Browse Source

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 years ago
parent
commit
a9110e1788
100 changed files with 4128 additions and 993 deletions
  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>
         <include>*-sources.jar</include>
       </includes>
       </includes>
     </fileSet>
     </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>
     <fileSet>
       <directory>hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/target</directory>
       <directory>hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/target</directory>
       <outputDirectory>/share/hadoop/${hadoop.component}/sources</outputDirectory>
       <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.
     HADOOP-9420. Add percentile or max metric for rpcQueueTime, processing time.
     (Liang Xie via wang)
     (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
   OPTIMIZATIONS
 
 
     HADOOP-9748. Reduce blocking on UGI.ensureInitialized (daryn)
     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
     HADOOP-9817. FileSystem#globStatus and FileContext#globStatus need to work
     with symlinks. (Colin Patrick McCabe via Andrew Wang)
     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
     HADOOP-9875.  TestDoAsEffectiveUser can fail on JDK 7.  (Aaron T. Myers via
     Colin Patrick McCabe)
     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-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
 Release 2.3.0 - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
@@ -550,6 +556,9 @@ Release 2.3.0 - UNRELEASED
     HADOOP-10132. RPC#stopProxy() should log the class of proxy when IllegalArgumentException 
     HADOOP-10132. RPC#stopProxy() should log the class of proxy when IllegalArgumentException 
     is encountered (Ted yu via umamahesh)
     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
   OPTIMIZATIONS
 
 
     HADOOP-10142. Avoid groups lookup for unprivileged users such as "dr.who"
     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-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
 Release 2.2.0 - 2013-10-13
 
 
   INCOMPATIBLE CHANGES
   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");
         "Property name must not be null");
     Preconditions.checkArgument(
     Preconditions.checkArgument(
         value != null,
         value != null,
-        "Property value must not be null");
+        "The value of property " + name + " must not be null");
     DeprecationContext deprecations = deprecationContext.get();
     DeprecationContext deprecations = deprecationContext.get();
     if (deprecations.getDeprecatedKeyMap().isEmpty()) {
     if (deprecations.getDeprecatedKeyMap().isEmpty()) {
       getProps();
       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.
  * limitations under the License.
  */
  */
 
 
+
 package org.apache.hadoop.fs;
 package org.apache.hadoop.fs;
 
 
+import com.google.common.annotations.VisibleForTesting;
+
 import java.io.BufferedOutputStream;
 import java.io.BufferedOutputStream;
 import java.io.DataOutput;
 import java.io.DataOutput;
 import java.io.File;
 import java.io.File;
@@ -51,7 +54,13 @@ import org.apache.hadoop.util.StringUtils;
 public class RawLocalFileSystem extends FileSystem {
 public class RawLocalFileSystem extends FileSystem {
   static final URI NAME = URI.create("file:///");
   static final URI NAME = URI.create("file:///");
   private Path workingDir;
   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() {
   public RawLocalFileSystem() {
     workingDir = getInitialWorkingDirectory();
     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,
   public HttpServer(String name, String bindAddress, int port,
       boolean findPort, Configuration conf, AccessControlList adminsAcl, 
       boolean findPort, Configuration conf, AccessControlList adminsAcl, 
       Connector connector, String[] pathSpecs) throws IOException {
       Connector connector, String[] pathSpecs) throws IOException {
-    this(new Builder().setName(name)
+    this(new Builder().setName(name).hostName(bindAddress)
         .addEndpoint(URI.create("http://" + bindAddress + ":" + port))
         .addEndpoint(URI.create("http://" + bindAddress + ":" + port))
         .setFindPort(findPort).setConf(conf).setACL(adminsAcl)
         .setFindPort(findPort).setConf(conf).setACL(adminsAcl)
         .setConnector(connector).setPathSpec(pathSpecs));
         .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.classification.InterfaceStability;
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.util.ReflectionUtils;
 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
 /** Factories for non-public writables.  Defining a factory permits {@link
  * ObjectWritable} to be able to construct instances of non-public classes. */
  * ObjectWritable} to be able to construct instances of non-public classes. */
 @InterfaceAudience.Public
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 @InterfaceStability.Stable
 public class WritableFactories {
 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
   private WritableFactories() {}                  // singleton
 
 
   /** Define a factory for a class. */
   /** 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);
     CLASS_TO_FACTORY.put(c, factory);
   }
   }
 
 
   /** Define a factory for a class. */
   /** 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);
     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 ");
       fail("Should throw an IllegalArgumentException exception ");
     } catch (Exception e) {
     } catch (Exception e) {
       assertTrue(e instanceof IllegalArgumentException);
       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 ");
       fail("Should throw an IllegalArgumentException exception ");
     } catch (Exception e) {
     } catch (Exception e) {
       assertTrue(e instanceof IllegalArgumentException);
       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.
  * Test symbolic links using LocalFs.
  */
  */
 abstract public class TestSymlinkLocalFS extends SymlinkBaseTest {
 abstract public class TestSymlinkLocalFS extends SymlinkBaseTest {
+
+  // Workaround for HADOOP-9652
+  static {
+    RawLocalFileSystem.useStatIfAvailable();
+  }
   
   
   @Override
   @Override
   protected String getScheme() {
   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));
     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 {
   @Test public void testBindAddress() throws Exception {
     checkBindAddress("localhost", 0, false).stop();
     checkBindAddress("localhost", 0, false).stop();
     // hang onto this one for a bit more testing
     // 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"/>
                     <delete file="${httpfs.tomcat.dist.dir}/conf/server.xml"/>
                     <copy file="${basedir}/src/main/tomcat/server.xml"
                     <copy file="${basedir}/src/main/tomcat/server.xml"
                           toDir="${httpfs.tomcat.dist.dir}/conf"/>
                           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"/>
                     <delete file="${httpfs.tomcat.dist.dir}/conf/logging.properties"/>
                     <copy file="${basedir}/src/main/tomcat/logging.properties"
                     <copy file="${basedir}/src/main/tomcat/logging.properties"
                           toDir="${httpfs.tomcat.dist.dir}/conf"/>
                           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
 # The hostname HttpFS server runs on
 #
 #
 # export HTTPFS_HTTP_HOSTNAME=`hostname -f`
 # 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) {
     if (makeQualified) {
       path = makeQualified(path);
       path = makeQualified(path);
     }
     }
-    final URL url = HttpFSUtils.createHttpURL(path, params);
+    final URL url = HttpFSUtils.createURL(path, params);
     return doAsRealUserIfNecessary(new Callable<HttpURLConnection>() {
     return doAsRealUserIfNecessary(new Callable<HttpURLConnection>() {
       @Override
       @Override
       public HttpURLConnection call() throws Exception {
       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>();
     Map<String, String> params = new HashMap<String, String>();
     params.put(OP_PARAM, op.toString());
     params.put(OP_PARAM, op.toString());
     params.put(RENEWER_PARAM,renewer);
     params.put(RENEWER_PARAM,renewer);
-    URL url = HttpFSUtils.createHttpURL(new Path(fsURI), params);
+    URL url = HttpFSUtils.createURL(new Path(fsURI), params);
     AuthenticatedURL aUrl =
     AuthenticatedURL aUrl =
       new AuthenticatedURL(new HttpFSKerberosAuthenticator());
       new AuthenticatedURL(new HttpFSKerberosAuthenticator());
     try {
     try {
@@ -150,7 +150,7 @@ public class HttpFSKerberosAuthenticator extends KerberosAuthenticator {
     params.put(OP_PARAM,
     params.put(OP_PARAM,
                DelegationTokenOperation.RENEWDELEGATIONTOKEN.toString());
                DelegationTokenOperation.RENEWDELEGATIONTOKEN.toString());
     params.put(TOKEN_PARAM, dToken.encodeToUrlString());
     params.put(TOKEN_PARAM, dToken.encodeToUrlString());
-    URL url = HttpFSUtils.createHttpURL(new Path(fsURI), params);
+    URL url = HttpFSUtils.createURL(new Path(fsURI), params);
     AuthenticatedURL aUrl =
     AuthenticatedURL aUrl =
       new AuthenticatedURL(new HttpFSKerberosAuthenticator());
       new AuthenticatedURL(new HttpFSKerberosAuthenticator());
     try {
     try {
@@ -172,7 +172,7 @@ public class HttpFSKerberosAuthenticator extends KerberosAuthenticator {
     params.put(OP_PARAM,
     params.put(OP_PARAM,
                DelegationTokenOperation.CANCELDELEGATIONTOKEN.toString());
                DelegationTokenOperation.CANCELDELEGATIONTOKEN.toString());
     params.put(TOKEN_PARAM, dToken.encodeToUrlString());
     params.put(TOKEN_PARAM, dToken.encodeToUrlString());
-    URL url = HttpFSUtils.createHttpURL(new Path(fsURI), params);
+    URL url = HttpFSUtils.createURL(new Path(fsURI), params);
     AuthenticatedURL aUrl =
     AuthenticatedURL aUrl =
       new AuthenticatedURL(new HttpFSKerberosAuthenticator());
       new AuthenticatedURL(new HttpFSKerberosAuthenticator());
     try {
     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,
    * @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 {
     throws IOException {
     URI uri = path.toUri();
     URI uri = path.toUri();
     String realScheme;
     String realScheme;
     if (uri.getScheme().equalsIgnoreCase(HttpFSFileSystem.SCHEME)) {
     if (uri.getScheme().equalsIgnoreCase(HttpFSFileSystem.SCHEME)) {
       realScheme = "http";
       realScheme = "http";
+    } else if (uri.getScheme().equalsIgnoreCase(HttpsFSFileSystem.SCHEME)) {
+      realScheme = "https";
+
     } else {
     } else {
       throw new IllegalArgumentException(MessageFormat.format(
       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();
     StringBuilder sb = new StringBuilder();
     sb.append(realScheme).append("://").append(uri.getAuthority()).
     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
   @Override
   public void init() throws ServerException {
   public void init() throws ServerException {
-    super.init();
     if (SERVER != null) {
     if (SERVER != null) {
       throw new RuntimeException("HttpFSServer server already initialized");
       throw new RuntimeException("HttpFSServer server already initialized");
     }
     }
     SERVER = this;
     SERVER = this;
+    super.init();
     adminGroup = getConfig().get(getPrefixedName(CONF_ADMIN_GROUP), "admin");
     adminGroup = getConfig().get(getPrefixedName(CONF_ADMIN_GROUP), "admin");
     LOG.info("Connects to Namenode [{}]",
     LOG.info("Connects to Namenode [{}]",
              get().get(FileSystemAccess.class).getFileSystemConfiguration().
              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
 public class DelegationTokenIdentifier
   extends AbstractDelegationTokenIdentifier {
   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
    * Create a new delegation token identifier
    *
    *
+   * @param kind token kind
    * @param owner the effective username of the token owner
    * @param owner the effective username of the token owner
    * @param renewer the username of the renewer
    * @param renewer the username of the renewer
    * @param realUser the real username of the token owner
    * @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);
     super(owner, renewer, realUser);
+    this.kind = kind;
   }
   }
 
 
-
   /**
   /**
    * Returns the kind, <code>TOKEN_KIND</code>.
    * Returns the kind, <code>TOKEN_KIND</code>.
    * @return returns <code>TOKEN_KIND</code>.
    * @return returns <code>TOKEN_KIND</code>.
    */
    */
   @Override
   @Override
   public Text getKind() {
   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.classification.InterfaceAudience;
 import org.apache.hadoop.fs.http.server.HttpFSServerWebApp;
 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.io.Text;
 import org.apache.hadoop.lib.server.BaseService;
 import org.apache.hadoop.lib.server.BaseService;
 import org.apache.hadoop.lib.server.ServerException;
 import org.apache.hadoop.lib.server.ServerException;
@@ -55,6 +57,8 @@ public class DelegationTokenManagerService extends BaseService
 
 
   DelegationTokenSecretManager secretManager = null;
   DelegationTokenSecretManager secretManager = null;
 
 
+  private Text tokenKind;
+
   public DelegationTokenManagerService() {
   public DelegationTokenManagerService() {
     super(PREFIX);
     super(PREFIX);
   }
   }
@@ -70,7 +74,9 @@ public class DelegationTokenManagerService extends BaseService
     long updateInterval = getServiceConfig().getLong(UPDATE_INTERVAL, DAY);
     long updateInterval = getServiceConfig().getLong(UPDATE_INTERVAL, DAY);
     long maxLifetime = getServiceConfig().getLong(MAX_LIFETIME, 7 * DAY);
     long maxLifetime = getServiceConfig().getLong(MAX_LIFETIME, 7 * DAY);
     long renewInterval = getServiceConfig().getLong(RENEW_INTERVAL, 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,
                                                      maxLifetime,
                                                      renewInterval, HOUR);
                                                      renewInterval, HOUR);
     try {
     try {
@@ -122,7 +128,7 @@ public class DelegationTokenManagerService extends BaseService
       realUser = new Text(ugi.getRealUser().getUserName());
       realUser = new Text(ugi.getRealUser().getUserName());
     }
     }
     DelegationTokenIdentifier tokenIdentifier =
     DelegationTokenIdentifier tokenIdentifier =
-      new DelegationTokenIdentifier(owner, new Text(renewer), realUser);
+      new DelegationTokenIdentifier(tokenKind, owner, new Text(renewer), realUser);
     Token<DelegationTokenIdentifier> token =
     Token<DelegationTokenIdentifier> token =
       new Token<DelegationTokenIdentifier>(tokenIdentifier, secretManager);
       new Token<DelegationTokenIdentifier>(tokenIdentifier, secretManager);
     try {
     try {
@@ -188,7 +194,7 @@ public class DelegationTokenManagerService extends BaseService
     throws DelegationTokenManagerException {
     throws DelegationTokenManagerException {
     ByteArrayInputStream buf = new ByteArrayInputStream(token.getIdentifier());
     ByteArrayInputStream buf = new ByteArrayInputStream(token.getIdentifier());
     DataInputStream dis = new DataInputStream(buf);
     DataInputStream dis = new DataInputStream(buf);
-    DelegationTokenIdentifier id = new DelegationTokenIdentifier();
+    DelegationTokenIdentifier id = new DelegationTokenIdentifier(tokenKind);
     try {
     try {
       id.readFields(dis);
       id.readFields(dis);
       dis.close();
       dis.close();
@@ -203,6 +209,8 @@ public class DelegationTokenManagerService extends BaseService
   private static class DelegationTokenSecretManager
   private static class DelegationTokenSecretManager
     extends AbstractDelegationTokenSecretManager<DelegationTokenIdentifier> {
     extends AbstractDelegationTokenSecretManager<DelegationTokenIdentifier> {
 
 
+    private Text tokenKind;
+
     /**
     /**
      * Create a secret manager
      * Create a secret manager
      *
      *
@@ -215,17 +223,18 @@ public class DelegationTokenManagerService extends BaseService
      * scanned
      * scanned
      * for expired tokens
      * for expired tokens
      */
      */
-    public DelegationTokenSecretManager(long delegationKeyUpdateInterval,
+    public DelegationTokenSecretManager(Text tokenKind, long delegationKeyUpdateInterval,
                                         long delegationTokenMaxLifetime,
                                         long delegationTokenMaxLifetime,
                                         long delegationTokenRenewInterval,
                                         long delegationTokenRenewInterval,
                                         long delegationTokenRemoverScanInterval) {
                                         long delegationTokenRemoverScanInterval) {
       super(delegationKeyUpdateInterval, delegationTokenMaxLifetime,
       super(delegationKeyUpdateInterval, delegationTokenMaxLifetime,
             delegationTokenRenewInterval, delegationTokenRemoverScanInterval);
             delegationTokenRenewInterval, delegationTokenRemoverScanInterval);
+      this.tokenKind = tokenKind;
     }
     }
 
 
     @Override
     @Override
     public DelegationTokenIdentifier createIdentifier() {
     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 TEMP_DIR = ".temp.dir";
   private static final String HTTP_HOSTNAME = ".http.hostname";
   private static final String HTTP_HOSTNAME = ".http.hostname";
   private static final String HTTP_PORT = ".http.port";
   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>();
   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) {
   public void setAuthority(InetSocketAddress authority) {
     this.authority = 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}"
   print "Using   HTTPFS_HTTP_HOSTNAME: ${HTTPFS_HTTP_HOSTNAME}"
 fi
 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
 if [ "${CATALINA_BASE}" = "" ]; then
   export CATALINA_BASE=${HTTPFS_HOME}/share/hadoop/httpfs/tomcat
   export CATALINA_BASE=${HTTPFS_HOME}/share/hadoop/httpfs/tomcat
   print "Setting CATALINA_BASE:       ${CATALINA_BASE}"
   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.admin.port=${HTTPFS_ADMIN_PORT}";
 catalina_opts="${catalina_opts} -Dhttpfs.http.port=${HTTPFS_HTTP_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.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}"
 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}}
   HttpFS supports the following {{{./httpfs-default.html}configuration properties}}
   in the HttpFS's <<<conf/httpfs-site.xml>>> configuration file.
   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}} \]
   \[ {{{./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;
     return HttpFSFileSystem.class;
   }
   }
 
 
+  protected String getScheme() {
+    return "webhdfs";
+  }
+
   protected FileSystem getHttpFSFileSystem() throws Exception {
   protected FileSystem getHttpFSFileSystem() throws Exception {
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
     conf.set("fs.webhdfs.impl", getFileSystemClass().getName());
     conf.set("fs.webhdfs.impl", getFileSystemClass().getName());
-    URI uri = new URI("webhdfs://" +
+    URI uri = new URI(getScheme() + "://" +
                       TestJettyHelper.getJettyURL().toURI().getAuthority());
                       TestJettyHelper.getJettyURL().toURI().getAuthority());
     return FileSystem.get(uri, conf);
     return FileSystem.get(uri, conf);
   }
   }
@@ -127,7 +131,7 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
   protected void testGet() throws Exception {
   protected void testGet() throws Exception {
     FileSystem fs = getHttpFSFileSystem();
     FileSystem fs = getHttpFSFileSystem();
     Assert.assertNotNull(fs);
     Assert.assertNotNull(fs);
-    URI uri = new URI("webhdfs://" +
+    URI uri = new URI(getScheme() + "://" +
                       TestJettyHelper.getJettyURL().toURI().getAuthority());
                       TestJettyHelper.getJettyURL().toURI().getAuthority());
     Assert.assertEquals(fs.getUri(), uri);
     Assert.assertEquals(fs.getUri(), uri);
     fs.close();
     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.HttpFSFileSystem;
 import org.apache.hadoop.fs.http.client.HttpFSKerberosAuthenticator;
 import org.apache.hadoop.fs.http.client.HttpFSKerberosAuthenticator;
 import org.apache.hadoop.fs.http.client.HttpFSKerberosAuthenticator.DelegationTokenOperation;
 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.DelegationTokenIdentifier;
 import org.apache.hadoop.lib.service.DelegationTokenManager;
 import org.apache.hadoop.lib.service.DelegationTokenManager;
 import org.apache.hadoop.lib.service.DelegationTokenManagerException;
 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.UserGroupInformation;
 import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.security.authentication.server.AuthenticationHandler;
 import org.apache.hadoop.security.authentication.server.AuthenticationHandler;
@@ -51,7 +55,24 @@ public class TestHttpFSKerberosAuthenticationHandler extends HFSTestCase {
 
 
   @Test
   @Test
   @TestDir
   @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();
     String dir = TestDirHelper.getTestDir().getAbsolutePath();
 
 
     Configuration httpfsConf = new Configuration(false);
     Configuration httpfsConf = new Configuration(false);
@@ -67,8 +88,8 @@ public class TestHttpFSKerberosAuthenticationHandler extends HFSTestCase {
 
 
       testNonManagementOperation(handler);
       testNonManagementOperation(handler);
       testManagementOperationErrors(handler);
       testManagementOperationErrors(handler);
-      testGetToken(handler, null);
-      testGetToken(handler, "foo");
+      testGetToken(handler, null, expectedTokenKind);
+      testGetToken(handler, "foo", expectedTokenKind);
       testCancelToken(handler);
       testCancelToken(handler);
       testRenewToken(handler);
       testRenewToken(handler);
 
 
@@ -112,8 +133,8 @@ public class TestHttpFSKerberosAuthenticationHandler extends HFSTestCase {
       Mockito.contains("requires SPNEGO"));
       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;
     DelegationTokenOperation op = DelegationTokenOperation.GETDELEGATIONTOKEN;
     HttpServletRequest request = Mockito.mock(HttpServletRequest.class);
     HttpServletRequest request = Mockito.mock(HttpServletRequest.class);
     HttpServletResponse response = Mockito.mock(HttpServletResponse.class);
     HttpServletResponse response = Mockito.mock(HttpServletResponse.class);
@@ -154,6 +175,7 @@ public class TestHttpFSKerberosAuthenticationHandler extends HFSTestCase {
     Token<DelegationTokenIdentifier> dt = new Token<DelegationTokenIdentifier>();
     Token<DelegationTokenIdentifier> dt = new Token<DelegationTokenIdentifier>();
     dt.decodeFromUrlString(tokenStr);
     dt.decodeFromUrlString(tokenStr);
     HttpFSServerWebApp.get().get(DelegationTokenManager.class).verifyToken(dt);
     HttpFSServerWebApp.get().get(DelegationTokenManager.class).verifyToken(dt);
+    Assert.assertEquals(expectedTokenKind, dt.getKind());
   }
   }
 
 
   private void testCancelToken(AuthenticationHandler handler)
   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.server.Server;
 import org.apache.hadoop.lib.service.DelegationTokenManager;
 import org.apache.hadoop.lib.service.DelegationTokenManager;
 import org.apache.hadoop.lib.service.DelegationTokenManagerException;
 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.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.test.HTestCase;
 import org.apache.hadoop.test.HTestCase;
@@ -43,9 +46,12 @@ public class TestDelegationTokenManagerService extends HTestCase {
   public void service() throws Exception {
   public void service() throws Exception {
     String dir = TestDirHelper.getTestDir().getAbsolutePath();
     String dir = TestDirHelper.getTestDir().getAbsolutePath();
     Configuration conf = new Configuration(false);
     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();
     server.init();
     DelegationTokenManager tm = server.get(DelegationTokenManager.class);
     DelegationTokenManager tm = server.get(DelegationTokenManager.class);
     Assert.assertNotNull(tm);
     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.rules.MethodRule;
 import org.junit.runners.model.FrameworkMethod;
 import org.junit.runners.model.FrameworkMethod;
 import org.junit.runners.model.Statement;
 import org.junit.runners.model.Statement;
+import org.mortbay.jetty.Connector;
 import org.mortbay.jetty.Server;
 import org.mortbay.jetty.Server;
+import org.mortbay.jetty.security.SslSocketConnector;
 
 
 public class TestJettyHelper implements MethodRule {
 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
   @Override
   public Statement apply(final Statement statement, final FrameworkMethod frameworkMethod, final Object o) {
   public Statement apply(final Statement statement, final FrameworkMethod frameworkMethod, final Object o) {
     return new Statement() {
     return new Statement() {
       @Override
       @Override
       public void evaluate() throws Throwable {
       public void evaluate() throws Throwable {
-        Server server = null;
         TestJetty testJetty = frameworkMethod.getAnnotation(TestJetty.class);
         TestJetty testJetty = frameworkMethod.getAnnotation(TestJetty.class);
         if (testJetty != null) {
         if (testJetty != null) {
           server = createJettyServer();
           server = createJettyServer();
         }
         }
         try {
         try {
-          TEST_SERVLET_TL.set(server);
+          TEST_JETTY_TL.set(TestJettyHelper.this);
           statement.evaluate();
           statement.evaluate();
         } finally {
         } finally {
-          TEST_SERVLET_TL.remove();
+          TEST_JETTY_TL.remove();
           if (server != null && server.isRunning()) {
           if (server != null && server.isRunning()) {
             try {
             try {
               server.stop();
               server.stop();
@@ -73,8 +88,19 @@ public class TestJettyHelper implements MethodRule {
       int port = ss.getLocalPort();
       int port = ss.getLocalPort();
       ss.close();
       ss.close();
       Server server = new Server(0);
       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;
       return server;
     } catch (Exception ex) {
     } catch (Exception ex) {
       throw new RuntimeException("Could not start embedded servlet container, " + ex.getMessage(), 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.
    * @return a Jetty server ready to be configured and the started.
    */
    */
   public static Server getJettyServer() {
   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");
       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.
    * @return the base URL (SCHEMA://HOST:PORT) of the test Jetty server.
    */
    */
   public static URL getJettyURL() {
   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");
       throw new IllegalStateException("This test does not use @TestJetty");
     }
     }
     try {
     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) {
     } catch (MalformedURLException ex) {
       throw new RuntimeException("It should never happen, " + ex.getMessage(), 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
     HDFS-5041. Add the time of last heartbeat to dead server Web UI (Shinichi
     Yamashita via brandonli)
     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.
     HDFS-5531. Combine the getNsQuota() and getDsQuota() methods in INode.
     (szetszwo)
     (szetszwo)
 
 
-    HDFS-5473. Consistent naming of user-visible caching classes and methods
-    (cmccabe)
-
     HDFS-5285. Flatten INodeFile hierarchy: Replace INodeFileUnderConstruction
     HDFS-5285. Flatten INodeFile hierarchy: Replace INodeFileUnderConstruction
     and INodeFileUnderConstructionWithSnapshot with FileUnderContructionFeature.
     and INodeFileUnderConstructionWithSnapshot with FileUnderContructionFeature.
     (jing9 via szetszwo)
     (jing9 via szetszwo)
@@ -215,15 +130,8 @@ Trunk (Unreleased)
     HDFS-5286. Flatten INodeDirectory hierarchy: Replace INodeDirectoryWithQuota
     HDFS-5286. Flatten INodeDirectory hierarchy: Replace INodeDirectoryWithQuota
     with DirectoryWithQuotaFeature.  (szetszwo)
     with DirectoryWithQuotaFeature.  (szetszwo)
 
 
-    HDFS-5556. Add some more NameNode cache statistics, cache pool stats
-    (cmccabe)
-
     HDFS-5537. Remove FileWithSnapshot interface.  (jing9 via szetszwo)
     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
     HDFS-5554. Flatten INodeFile hierarchy: Replace INodeFileWithSnapshot with
     FileWithSnapshotFeature.  (jing9 via szetszwo)
     FileWithSnapshotFeature.  (jing9 via szetszwo)
 
 
@@ -234,26 +142,15 @@ Trunk (Unreleased)
     INodeDirectoryWithSnapshot with DirectoryWithSnapshotFeature.
     INodeDirectoryWithSnapshot with DirectoryWithSnapshotFeature.
     (jing9 via szetszwo)
     (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
     HDFS-5715. Use Snapshot ID to indicate the corresponding Snapshot for a
     FileDiff/DirectoryDiff. (jing9)
     FileDiff/DirectoryDiff. (jing9)
 
 
     HDFS-5721. sharedEditsImage in Namenode#initializeSharedEdits() should be 
     HDFS-5721. sharedEditsImage in Namenode#initializeSharedEdits() should be 
     closed before method returns. (Ted Yu via junping_du)
     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
   BUG FIXES
 
 
@@ -372,110 +269,12 @@ Trunk (Unreleased)
     HDFS-4366. Block Replication Policy Implementation May Skip Higher-Priority
     HDFS-4366. Block Replication Policy Implementation May Skip Higher-Priority
     Blocks for Lower-Priority Blocks (Derek Dagit via kihwal)
     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
     HDFS-5705. TestSecondaryNameNodeUpgrade#testChangeNsIDFails may fail due
     to ConcurrentModificationException. (Ted Yu via brandonli)
     to ConcurrentModificationException. (Ted Yu via brandonli)
 
 
     HDFS-5719. FSImage#doRollback() should close prevState before return
     HDFS-5719. FSImage#doRollback() should close prevState before return
     (Ted Yu via brandonli)
     (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-5726. Fix compilation error in AbstractINodeDiff for JDK7. (jing9)
 
 
     HDFS-5768. Consolidate the serialization code in DelegationTokenSecretManager 
     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
     HDFS-5784. reserve space in edit log header and fsimage header for feature
     flag section (cmccabe)
     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
   IMPROVEMENTS
 
 
     HDFS-5267. Remove volatile from LightWeightHashSet. (Junping Du via llu)
     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-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
   OPTIMIZATIONS
 
 
     HDFS-5239.  Allow FSNamesystem lock fairness to be configurable (daryn)
     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-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
   BUG FIXES
 
 
     HDFS-5034.  Remove debug prints from GetFileLinkInfo (Andrew Wang via Colin
     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-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
   BREAKDOWN OF HDFS-2832 SUBTASKS AND RELATED JIRAS
 
 
     HDFS-4985. Add storage type to the protocol and expose it in block report
     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)
     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
 Release 2.3.0 - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
@@ -1108,6 +1137,11 @@ Release 2.3.0 - UNRELEASED
     HDFS-5649. Unregister NFS and Mount service when NFS gateway is shutting down.
     HDFS-5649. Unregister NFS and Mount service when NFS gateway is shutting down.
     (brandonli)
     (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
 Release 2.2.0 - 2013-10-13
 
 
   INCOMPATIBLE CHANGES
   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 
     HDFS-5091. Support for spnego keytab separate from the JournalNode keytab 
     for secure HA. (jing9)
     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)
     setups. (Vinay and suresh via suresh)
 
 
     HDFS-4898. BlockPlacementPolicyWithNodeGroup.chooseRemoteRack() fails to
     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" />
       <Class name="org.apache.hadoop.hdfs.server.blockmanagement.CacheReplicationMonitor" />
       <Bug pattern="RV_RETURN_VALUE_IGNORED_BAD_PRACTICE" />
       <Bug pattern="RV_RETURN_VALUE_IGNORED_BAD_PRACTICE" />
     </Match>
     </Match>
+    <Match>
+      <Class name="org.apache.hadoop.hdfs.DFSUtil"/>
+      <Method name="assertAllResultsEqual" />
+      <Bug pattern="NP_LOAD_OF_KNOWN_NULL_VALUE" />
+    </Match>
 
 
  </FindBugsFilter>
  </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;
 package org.apache.hadoop.contrib.bkjournal;
 
 
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 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.JournalManager;
 import org.apache.hadoop.hdfs.server.namenode.EditLogOutputStream;
 import org.apache.hadoop.hdfs.server.namenode.EditLogOutputStream;
 import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream;
 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
   @Override
   public void close() throws IOException {
   public void close() throws IOException {
     try {
     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) {
     } catch (IOException ioe) {
       LOG.info("Got expected exception", ioe);
       LOG.info("Got expected exception", ioe);
       GenericTestUtils.assertExceptionContains(
       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();
         final long elapsed = Time.now() - getLastLeaseRenewal();
         if (elapsed > HdfsConstants.LEASE_HARDLIMIT_PERIOD) {
         if (elapsed > HdfsConstants.LEASE_HARDLIMIT_PERIOD) {
           LOG.warn("Failed to renew lease for " + clientName + " for "
           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.) "
               + (HdfsConstants.LEASE_HARDLIMIT_PERIOD/1000) + " seconds.) "
               + "Closing all files being written ...", e);
               + "Closing all files being written ...", e);
           closeAllFilesBeingWritten(true);
           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_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 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 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.
   // 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";
   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);
             currentNode = blockSeekTo(pos);
           }
           }
           int realLen = (int) Math.min(len, (blockEnd - pos + 1L));
           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);
           int result = readBuffer(strategy, off, realLen, corruptedBlockMap);
           
           
           if (result >= 0) {
           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.security.SecureRandom;
 import java.text.SimpleDateFormat;
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.Comparator;
@@ -574,10 +575,24 @@ public class DFSUtil {
     }
     }
     return ret;
     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(
   private static Map<String, InetSocketAddress> getAddressesForNameserviceId(
       Configuration conf, String nsId, String defaultValue,
       Configuration conf, String nsId, String defaultValue,
-      String[] keys) {
+      String... keys) {
     Collection<String> nnIds = getNameNodeIds(conf, nsId);
     Collection<String> nnIds = getNameNodeIds(conf, nsId);
     Map<String, InetSocketAddress> ret = Maps.newHashMap();
     Map<String, InetSocketAddress> ret = Maps.newHashMap();
     for (String nnId : emptyAsSingletonNull(nnIds)) {
     for (String nnId : emptyAsSingletonNull(nnIds)) {
@@ -1670,4 +1685,32 @@ public class DFSUtil {
     }
     }
     return builder;
     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.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 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.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.RPC;
 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.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URISyntaxException;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collection;
+import java.util.List;
 import java.util.Map;
 import java.util.Map;
 
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
@@ -305,4 +312,55 @@ public class HAUtil {
     DFSClient dfsClient = dfs.getClient();
     DFSClient dfsClient = dfs.getClient();
     return RPC.getServerAddress(dfsClient.getNamenode());
     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.PrepareRecoveryResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.SegmentStateProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.SegmentStateProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.RequestInfo;
 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.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
 
 
@@ -151,4 +152,17 @@ interface AsyncLogger {
    * StringBuilder. This is displayed on the NN web UI.
    * StringBuilder. This is displayed on the NN web UI.
    */
    */
   public void appendReport(StringBuilder sb);
   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.NewEpochResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryResponseProto;
 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.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.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
 
 
@@ -308,4 +309,71 @@ class AsyncLoggerSet {
     }
     }
     return QuorumCall.create(calls);
     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.QJournalProtocolPB;
 import org.apache.hadoop.hdfs.qjournal.protocolPB.QJournalProtocolTranslatorPB;
 import org.apache.hadoop.hdfs.qjournal.protocolPB.QJournalProtocolTranslatorPB;
 import org.apache.hadoop.hdfs.qjournal.server.GetJournalEditServlet;
 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.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 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
   @Override
   public String toString() {
   public String toString() {
@@ -636,4 +703,5 @@ public class IPCLoggerChannel implements AsyncLogger {
   private boolean hasHttpServerEndPoint() {
   private boolean hasHttpServerEndPoint() {
    return httpServerURL != null;
    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.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 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.GetJournalStateResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochResponseProto;
 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.PrepareRecoveryResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.SegmentStateProto;
 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.EditLogFileInputStream;
 import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream;
 import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream;
 import org.apache.hadoop.hdfs.server.namenode.EditLogOutputStream;
 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
   // Since these don't occur during normal operation, we can
   // use rather lengthy timeouts, and don't need to make them
   // use rather lengthy timeouts, and don't need to make them
   // configurable.
   // 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 Configuration conf;
   private final URI uri;
   private final URI uri;
@@ -492,4 +501,131 @@ public class QuorumJournalManager implements JournalManager {
     return loggers;
     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.PrepareRecoveryResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.SegmentStateProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.SegmentStateProto;
 import org.apache.hadoop.hdfs.qjournal.server.JournalNode;
 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.namenode.JournalManager;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.security.KerberosInfo;
 import org.apache.hadoop.security.KerberosInfo;
@@ -143,4 +144,17 @@ public interface QJournalProtocol {
    */
    */
   public void acceptRecovery(RequestInfo reqInfo,
   public void acceptRecovery(RequestInfo reqInfo,
       SegmentStateProto stateToAccept, URL fromUrl) throws IOException;
       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;
 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.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 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.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocol;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocol;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos;
 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.FinalizeLogSegmentRequestProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.FinalizeLogSegmentResponseProto;
 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.FormatRequestProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.FormatResponseProto;
 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.GetEditLogManifestRequestProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetEditLogManifestResponseProto;
 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.GetJournalStateRequestProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournalStateResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournalStateResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.HeartbeatRequestProto;
 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.JournalResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochRequestProto;
 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.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.PrepareRecoveryRequestProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PurgeLogsRequestProto;
 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.StartLogSegmentRequestProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.StartLogSegmentResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.StartLogSegmentResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.RequestInfo;
 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.RpcController;
 import com.google.protobuf.ServiceException;
 import com.google.protobuf.ServiceException;
 
 
-import java.io.IOException;
-import java.net.URL;
-
 /**
 /**
  * Implementation for protobuf service that forwards requests
  * Implementation for protobuf service that forwards requests
  * received on {@link JournalProtocolPB} to the 
  * received on {@link JournalProtocolPB} to the 
@@ -244,4 +258,79 @@ public class QJournalProtocolServerSideTranslatorPB implements QJournalProtocolP
         reqInfo.hasCommittedTxId() ?
         reqInfo.hasCommittedTxId() ?
           reqInfo.getCommittedTxId() : HdfsConstants.INVALID_TXID);
           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.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 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.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocol;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocol;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos;
 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.FinalizeLogSegmentRequestProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.FormatRequestProto;
 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.GetEditLogManifestRequestProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetEditLogManifestResponseProto;
 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.GetJournalStateRequestProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournalStateResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournalStateResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.HeartbeatRequestProto;
 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.JournalRequestProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochRequestProto;
 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.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.PrepareRecoveryRequestProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PurgeLogsRequestProto;
 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.SegmentStateProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.StartLogSegmentRequestProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.StartLogSegmentRequestProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.RequestInfo;
 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.JournalProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtobufHelper;
@@ -277,4 +287,85 @@ public class QJournalProtocolTranslatorPB implements ProtocolMetaInterface,
         RPC.getProtocolVersion(QJournalProtocolPB.class), methodName);
         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.DFSUtil;
 import org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager;
 import org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
 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;
 import org.apache.hadoop.hdfs.server.namenode.FileJournalManager.EditLogFile;
 import org.apache.hadoop.hdfs.server.namenode.FileJournalManager.EditLogFile;
 import org.apache.hadoop.hdfs.server.namenode.GetImageServlet;
 import org.apache.hadoop.hdfs.server.namenode.GetImageServlet;
@@ -139,20 +140,26 @@ public class GetJournalEditServlet extends HttpServlet {
   private boolean checkStorageInfoOrSendError(JNStorage storage,
   private boolean checkStorageInfoOrSendError(JNStorage storage,
       HttpServletRequest request, HttpServletResponse response)
       HttpServletRequest request, HttpServletResponse response)
       throws IOException {
       throws IOException {
-    String myStorageInfoString = storage.toColonSeparatedString();
+    int myNsId = storage.getNamespaceID();
+    String myClusterId = storage.getClusterID();
+    
     String theirStorageInfoString = StringEscapeUtils.escapeHtml(
     String theirStorageInfoString = StringEscapeUtils.escapeHtml(
         request.getParameter(STORAGEINFO_PARAM));
         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;
     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");
     return new File(sd.getCurrentDir(), "paxos");
   }
   }
   
   
+  File getRoot() {
+    return sd.getRoot();
+  }
+  
   /**
   /**
    * Remove any log files and associated paxos files which are older than
    * Remove any log files and associated paxos files which are older than
    * the given txid.
    * the given txid.
@@ -182,12 +186,15 @@ class JNStorage extends Storage {
     unlockAll();
     unlockAll();
     sd.clearDirectory();
     sd.clearDirectory();
     writeProperties(sd);
     writeProperties(sd);
+    createPaxosDir();
+    analyzeStorage();
+  }
+  
+  void createPaxosDir() throws IOException {
     if (!getPaxosDir().mkdirs()) {
     if (!getPaxosDir().mkdirs()) {
       throw new IOException("Could not create paxos dir: " + getPaxosDir());
       throw new IOException("Could not create paxos dir: " + getPaxosDir());
     }
     }
-    analyzeStorage();
   }
   }
-
   
   
   void analyzeStorage() throws IOException {
   void analyzeStorage() throws IOException {
     this.state = sd.analyzeStorage(StartupOption.REGULAR, this);
     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.JournalNotFormattedException;
 import org.apache.hadoop.hdfs.qjournal.protocol.JournalOutOfSyncException;
 import org.apache.hadoop.hdfs.qjournal.protocol.JournalOutOfSyncException;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocol;
 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.NewEpochResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PersistedRecoveryPaxosData;
 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.PrepareRecoveryResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.SegmentStateProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.SegmentStateProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.RequestInfo;
 import org.apache.hadoop.hdfs.qjournal.protocol.RequestInfo;
 import org.apache.hadoop.hdfs.server.common.StorageErrorReporter;
 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.EditLogOutputStream;
 import org.apache.hadoop.hdfs.server.namenode.FileJournalManager;
 import org.apache.hadoop.hdfs.server.namenode.FileJournalManager;
 import org.apache.hadoop.hdfs.server.namenode.FileJournalManager.EditLogFile;
 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
  * Each such journal is entirely independent despite being hosted by
  * the same JVM.
  * the same JVM.
  */
  */
-class Journal implements Closeable {
+public class Journal implements Closeable {
   static final Log LOG = LogFactory.getLog(Journal.class);
   static final Log LOG = LogFactory.getLog(Journal.class);
 
 
 
 
@@ -122,8 +124,8 @@ class Journal implements Closeable {
    */
    */
   private BestEffortLongFile committedTxnId;
   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 static final String COMMITTED_TXID_FILENAME = "committed-txid";
   
   
   private final FileJournalManager fjm;
   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,
   public RemoteEditLogManifest getEditLogManifest(long sinceTxId,
       boolean inProgressOk) throws IOException {
       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,
   public synchronized void acceptRecovery(RequestInfo reqInfo,
       SegmentStateProto segment, URL fromUrl)
       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.HdfsConfiguration;
 import org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager;
 import org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager;
 import org.apache.hadoop.hdfs.server.common.StorageErrorReporter;
 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.io.IOUtils;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.metrics2.source.JvmMetrics;
 import org.apache.hadoop.metrics2.source.JvmMetrics;
@@ -285,4 +286,31 @@ public class JournalNode implements Tool, Configurable, JournalNodeMXBean {
     StringUtils.startupShutdownMessage(JournalNode.class, args, LOG);
     StringUtils.startupShutdownMessage(JournalNode.class, args, LOG);
     System.exit(ToolRunner.run(new JournalNode(), args));
     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.protocol.RequestInfo;
 import org.apache.hadoop.hdfs.qjournal.protocolPB.QJournalProtocolPB;
 import org.apache.hadoop.hdfs.qjournal.protocolPB.QJournalProtocolPB;
 import org.apache.hadoop.hdfs.qjournal.protocolPB.QJournalProtocolServerSideTranslatorPB;
 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.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
@@ -205,4 +206,35 @@ class JournalNodeRpcServer implements QJournalProtocol {
         .acceptRecovery(reqInfo, log, fromUrl);
         .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(
         sock.connect(
             NetUtils.createSocketAddr(target.datanode.getXferAddr()),
             NetUtils.createSocketAddr(target.datanode.getXferAddr()),
             HdfsServerConstants.READ_TIMEOUT);
             HdfsServerConstants.READ_TIMEOUT);
+        sock.setSoTimeout(HdfsServerConstants.READ_TIMEOUT);
         sock.setKeepAlive(true);
         sock.setKeepAlive(true);
         
         
         OutputStream unbufOut = sock.getOutputStream();
         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;
   protected int tolerateHeartbeatMultiplier;
 
 
-  BlockPlacementPolicyDefault(Configuration conf,  FSClusterStats stats,
+  protected BlockPlacementPolicyDefault(Configuration conf, FSClusterStats stats,
                            NetworkTopology clusterMap) {
                            NetworkTopology clusterMap) {
     initialize(conf, stats, 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 {
 public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefault {
 
 
-  BlockPlacementPolicyWithNodeGroup(Configuration conf,  FSClusterStats stats,
+  protected BlockPlacementPolicyWithNodeGroup(Configuration conf,  FSClusterStats stats,
       NetworkTopology clusterMap) {
       NetworkTopology clusterMap) {
     initialize(conf, stats, clusterMap);
     initialize(conf, stats, clusterMap);
   }
   }
 
 
-  BlockPlacementPolicyWithNodeGroup() {
+  protected BlockPlacementPolicyWithNodeGroup() {
   }
   }
 
 
   @Override
   @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;
 package org.apache.hadoop.hdfs.server.common;
 
 
 import java.io.File;
 import java.io.File;
-import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.RandomAccessFile;
 import java.io.RandomAccessFile;
@@ -26,26 +25,23 @@ import java.lang.management.ManagementFactory;
 import java.nio.channels.FileLock;
 import java.nio.channels.FileLock;
 import java.nio.channels.OverlappingFileLockException;
 import java.nio.channels.OverlappingFileLockException;
 import java.util.ArrayList;
 import java.util.ArrayList;
-import java.util.List;
 import java.util.Iterator;
 import java.util.Iterator;
+import java.util.List;
 import java.util.Properties;
 import java.util.Properties;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 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.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 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.ToolRunner;
 import org.apache.hadoop.util.VersionInfo;
 import org.apache.hadoop.util.VersionInfo;
 
 
-import com.google.common.base.Preconditions;
-
 import com.google.common.base.Charsets;
 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 int[] LAYOUT_VERSIONS_203 = {-19, -31};
 
 
   public    static final String STORAGE_FILE_LOCK     = "in_use.lock";
   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_CURRENT   = "current";
   public    static final String STORAGE_DIR_PREVIOUS  = "previous";
   public    static final String STORAGE_DIR_PREVIOUS  = "previous";
   public    static final String STORAGE_TMP_REMOVED   = "removed.tmp";
   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> {
   private class DirIterator implements Iterator<StorageDirectory> {
     StorageDirType dirType;
     StorageDirType dirType;
+    boolean includeShared;
     int prevIndex; // for remove()
     int prevIndex; // for remove()
     int nextIndex; // for next()
     int nextIndex; // for next()
     
     
-    DirIterator(StorageDirType dirType) {
+    DirIterator(StorageDirType dirType, boolean includeShared) {
       this.dirType = dirType;
       this.dirType = dirType;
       this.nextIndex = 0;
       this.nextIndex = 0;
       this.prevIndex = 0;
       this.prevIndex = 0;
+      this.includeShared = includeShared;
     }
     }
     
     
     @Override
     @Override
     public boolean hasNext() {
     public boolean hasNext() {
       if (storageDirs.isEmpty() || nextIndex >= storageDirs.size())
       if (storageDirs.isEmpty() || nextIndex >= storageDirs.size())
         return false;
         return false;
-      if (dirType != null) {
+      if (dirType != null || !includeShared) {
         while (nextIndex < storageDirs.size()) {
         while (nextIndex < storageDirs.size()) {
-          if (getStorageDir(nextIndex).getStorageDirType().isOfType(dirType))
+          if (shouldReturnNextDir())
             break;
             break;
           nextIndex++;
           nextIndex++;
         }
         }
@@ -156,9 +153,9 @@ public abstract class Storage extends StorageInfo {
       StorageDirectory sd = getStorageDir(nextIndex);
       StorageDirectory sd = getStorageDir(nextIndex);
       prevIndex = nextIndex;
       prevIndex = nextIndex;
       nextIndex++;
       nextIndex++;
-      if (dirType != null) {
+      if (dirType != null || !includeShared) {
         while (nextIndex < storageDirs.size()) {
         while (nextIndex < storageDirs.size()) {
-          if (getStorageDir(nextIndex).getStorageDirType().isOfType(dirType))
+          if (shouldReturnNextDir())
             break;
             break;
           nextIndex++;
           nextIndex++;
         }
         }
@@ -172,6 +169,12 @@ public abstract class Storage extends StorageInfo {
       storageDirs.remove(prevIndex); // remove last returned element
       storageDirs.remove(prevIndex); // remove last returned element
       hasNext(); // reset nextIndex to correct place
       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
    * them via the Iterator
    */
    */
   public Iterator<StorageDirectory> dirIterator(StorageDirType dirType) {
   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) {
   public Iterable<StorageDirectory> dirIterable(final StorageDirType dirType) {
@@ -233,7 +256,9 @@ public abstract class Storage extends StorageInfo {
   @InterfaceAudience.Private
   @InterfaceAudience.Private
   public static class StorageDirectory implements FormatConfirmable {
   public static class StorageDirectory implements FormatConfirmable {
     final File root;              // root directory
     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
     final StorageDirType dirType; // storage dir type
     FileLock lock;                // storage lock
     FileLock lock;                // storage lock
 
 
@@ -241,11 +266,11 @@ public abstract class Storage extends StorageInfo {
     
     
     public StorageDirectory(File dir) {
     public StorageDirectory(File dir) {
       // default dirType is null
       // default dirType is null
-      this(dir, null, true);
+      this(dir, null, false);
     }
     }
     
     
     public StorageDirectory(File dir, StorageDirType dirType) {
     public StorageDirectory(File dir, StorageDirType dirType) {
-      this(dir, dirType, true);
+      this(dir, dirType, false);
     }
     }
     
     
     public void setStorageUuid(String storageUuid) {
     public void setStorageUuid(String storageUuid) {
@@ -260,14 +285,14 @@ public abstract class Storage extends StorageInfo {
      * Constructor
      * Constructor
      * @param dir directory corresponding to the storage
      * @param dir directory corresponding to the storage
      * @param dirType storage directory type
      * @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.root = dir;
       this.lock = null;
       this.lock = null;
       this.dirType = dirType;
       this.dirType = dirType;
-      this.useLock = useLock;
+      this.isShared = isShared;
     }
     }
     
     
     /**
     /**
@@ -621,6 +646,10 @@ public abstract class Storage extends StorageInfo {
       
       
       return true;
       return true;
     }
     }
+    
+    public boolean isShared() {
+      return isShared;
+    }
 
 
 
 
     /**
     /**
@@ -635,7 +664,7 @@ public abstract class Storage extends StorageInfo {
      * @throws IOException if locking fails
      * @throws IOException if locking fails
      */
      */
     public void lock() throws IOException {
     public void lock() throws IOException {
-      if (!useLock) {
+      if (isShared()) {
         LOG.info("Locking is disabled");
         LOG.info("Locking is disabled");
         return;
         return;
       }
       }
@@ -889,22 +918,6 @@ public abstract class Storage extends StorageInfo {
     public String toString();
     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.
    * Set common storage fields into the given properties object.
    * Should be overloaded if additional fields need to be set.
    * 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));
     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 {
   public void writeProperties(StorageDirectory sd) throws IOException {
     writeProperties(sd.getVersionFile(), sd);
     writeProperties(sd.getVersionFile(), sd);
   }
   }
-
+  
   public void writeProperties(File to, StorageDirectory sd) throws IOException {
   public void writeProperties(File to, StorageDirectory sd) throws IOException {
     Properties props = new Properties();
     Properties props = new Properties();
     setPropertiesFromFields(props, sd);
     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");
     RandomAccessFile file = new RandomAccessFile(to, "rws");
     FileOutputStream out = null;
     FileOutputStream out = null;
     try {
     try {
@@ -977,23 +1002,6 @@ public abstract class Storage extends StorageInfo {
       file.close();
       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 {
   public static void rename(File from, File to) throws IOException {
     if (!from.renameTo(to))
     if (!from.renameTo(to))
@@ -1044,69 +1052,6 @@ public abstract class Storage extends StorageInfo {
       + "-" + Long.toString(storage.getCTime());
       + "-" + 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) {
   public static boolean is203LayoutVersion(int layoutVersion) {
     for (int lv203 : LAYOUT_VERSIONS_203) {
     for (int lv203 : LAYOUT_VERSIONS_203) {
       if (lv203 == layoutVersion) {
       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;
 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.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
+import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 
 
 import com.google.common.base.Joiner;
 import com.google.common.base.Joiner;
 
 
@@ -34,6 +42,8 @@ public class StorageInfo {
   public int   namespaceID;     // id of the file system
   public int   namespaceID;     // id of the file system
   public String clusterID;      // id of the cluster
   public String clusterID;      // id of the cluster
   public long  cTime;           // creation time of the file system state
   public long  cTime;           // creation time of the file system state
+  
+  protected static final String STORAGE_FILE_VERSION    = "VERSION";
  
  
   public StorageInfo () {
   public StorageInfo () {
     this(0, 0, "", 0L);
     this(0, 0, "", 0L);
@@ -96,4 +106,113 @@ public class StorageInfo {
     return Joiner.on(":").join(
     return Joiner.on(":").join(
         layoutVersion, namespaceID, cTime, clusterID);
         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());
         dataDirs.size());
     for (Iterator<File> it = dataDirs.iterator(); it.hasNext();) {
     for (Iterator<File> it = dataDirs.iterator(); it.hasNext();) {
       File dataDir = it.next();
       File dataDir = it.next();
-      StorageDirectory sd = new StorageDirectory(dataDir, null, false);
+      StorageDirectory sd = new StorageDirectory(dataDir, null, true);
       StorageState curState;
       StorageState curState;
       try {
       try {
         curState = sd.analyzeStorage(startOpt, this);
         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.FileInputStream;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStream;
+import java.io.RandomAccessFile;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.DU;
 import org.apache.hadoop.fs.DU;
@@ -191,7 +192,7 @@ class BlockPoolSlice {
             blockFile.length(), genStamp, volume, blockFile.getParentFile());
             blockFile.length(), genStamp, volume, blockFile.getParentFile());
       } else {
       } else {
         newReplica = new ReplicaWaitingToBeRecovered(blockId,
         newReplica = new ReplicaWaitingToBeRecovered(blockId,
-            validateIntegrity(blockFile, genStamp), 
+            validateIntegrityAndSetLength(blockFile, genStamp), 
             genStamp, volume, blockFile.getParentFile());
             genStamp, volume, blockFile.getParentFile());
       }
       }
 
 
@@ -214,7 +215,7 @@ class BlockPoolSlice {
    * @param genStamp generation stamp of the block
    * @param genStamp generation stamp of the block
    * @return the number of valid bytes
    * @return the number of valid bytes
    */
    */
-  private long validateIntegrity(File blockFile, long genStamp) {
+  private long validateIntegrityAndSetLength(File blockFile, long genStamp) {
     DataInputStream checksumIn = null;
     DataInputStream checksumIn = null;
     InputStream blockIn = null;
     InputStream blockIn = null;
     try {
     try {
@@ -257,11 +258,25 @@ class BlockPoolSlice {
       IOUtils.readFully(blockIn, buf, 0, lastChunkSize);
       IOUtils.readFully(blockIn, buf, 0, lastChunkSize);
 
 
       checksum.update(buf, 0, lastChunkSize);
       checksum.update(buf, 0, lastChunkSize);
+      long validFileLength;
       if (checksum.compare(buf, lastChunkSize)) { // last chunk matches crc
       if (checksum.compare(buf, lastChunkSize)) { // last chunk matches crc
-        return lastChunkStartPos + lastChunkSize;
+        validFileLength = lastChunkStartPos + lastChunkSize;
       } else { // last chunck is corrupt
       } 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) {
     } catch (IOException e) {
       FsDatasetImpl.LOG.warn(e);
       FsDatasetImpl.LOG.warn(e);
       return 0;
       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.io.IOException;
 import java.util.Collection;
 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.JournalInfo;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
@@ -97,4 +99,35 @@ class BackupJournalManager implements JournalManager {
   public String toString() {
   public String toString() {
     return "BackupJournalManager";
     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.JournalProtocolPB;
 import org.apache.hadoop.hdfs.protocolPB.JournalProtocolServerSideTranslatorPB;
 import org.apache.hadoop.hdfs.protocolPB.JournalProtocolServerSideTranslatorPB;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 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.common.Storage;
 import org.apache.hadoop.hdfs.server.namenode.ha.HAState;
 import org.apache.hadoop.hdfs.server.namenode.ha.HAState;
 import org.apache.hadoop.hdfs.server.protocol.FenceResponse;
 import org.apache.hadoop.hdfs.server.protocol.FenceResponse;
@@ -367,7 +368,7 @@ public class BackupNode extends NameNode {
     } else {
     } else {
       nsInfo.validateStorage(storage);
       nsInfo.validateStorage(storage);
     }
     }
-    bnImage.initEditLog();
+    bnImage.initEditLog(StartupOption.REGULAR);
     setRegistration();
     setRegistration();
     NamenodeRegistration nnReg = null;
     NamenodeRegistration nnReg = null;
     while(!isStopRequested()) {
     while(!isStopRequested()) {
@@ -423,7 +424,8 @@ public class BackupNode extends NameNode {
     return DFSUtil.getBackupNameServiceId(conf);
     return DFSUtil.getBackupNameServiceId(conf);
   }
   }
 
 
-  protected HAState createHAState() {
+  @Override
+  protected HAState createHAState(StartupOption startOpt) {
     return new BackupState();
     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);
         DFSConfigKeys.DFS_LIST_LIMIT, DFSConfigKeys.DFS_LIST_LIMIT_DEFAULT);
     this.lsLimit = configuredLimit>0 ?
     this.lsLimit = configuredLimit>0 ?
         configuredLimit : DFSConfigKeys.DFS_LIST_LIMIT_DEFAULT;
         configuredLimit : DFSConfigKeys.DFS_LIST_LIMIT_DEFAULT;
-
     this.contentCountLimit = conf.getInt(
     this.contentCountLimit = conf.getInt(
         DFSConfigKeys.DFS_CONTENT_SUMMARY_LIMIT_KEY,
         DFSConfigKeys.DFS_CONTENT_SUMMARY_LIMIT_KEY,
         DFSConfigKeys.DFS_CONTENT_SUMMARY_LIMIT_DEFAULT);
         DFSConfigKeys.DFS_CONTENT_SUMMARY_LIMIT_DEFAULT);
@@ -1490,6 +1489,11 @@ public class FSDirectory implements Closeable {
   /**
   /**
    * Get a partial listing of the indicated directory
    * 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 src the directory name
    * @param startAfter the name to start listing after
    * @param startAfter the name to start listing after
    * @param needLocation if block locations are returned
    * @param needLocation if block locations are returned
@@ -1521,14 +1525,30 @@ public class FSDirectory implements Closeable {
       int startChild = INodeDirectory.nextChild(contents, startAfter);
       int startChild = INodeDirectory.nextChild(contents, startAfter);
       int totalNumChildren = contents.size();
       int totalNumChildren = contents.size();
       int numOfListing = Math.min(totalNumChildren-startChild, this.lsLimit);
       int numOfListing = Math.min(totalNumChildren-startChild, this.lsLimit);
+      int locationBudget = this.lsLimit;
+      int listingCnt = 0;
       HdfsFileStatus listing[] = new HdfsFileStatus[numOfListing];
       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);
         INode cur = contents.get(startChild+i);
         listing[i] = createFileStatus(cur.getLocalNameBytes(), cur,
         listing[i] = createFileStatus(cur.getLocalNameBytes(), cur,
             needLocation, snapshot);
             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(
       return new DirectoryListing(
-          listing, totalNumChildren-startChild-numOfListing);
+          listing, totalNumChildren-startChild-listingCnt);
     } finally {
     } finally {
       readUnlock();
       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.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 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.FormatConfirmable;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddBlockOp;
 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)) {
       if (u.getScheme().equals(NNStorage.LOCAL_URI_SCHEME)) {
         StorageDirectory sd = storage.getStorageDirectory(u);
         StorageDirectory sd = storage.getStorageDirectory(u);
         if (sd != null) {
         if (sd != null) {
-          journalSet.add(new FileJournalManager(conf, sd, storage), required);
+          journalSet.add(new FileJournalManager(conf, sd, storage),
+              required, sharedEditsDirs.contains(u));
         }
         }
       } else {
       } 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
   @Override
   public void selectInputStreams(Collection<EditLogInputStream> streams,
   public void selectInputStreams(Collection<EditLogInputStream> streams,
       long fromTxId, boolean inProgressOk) throws IOException {
       long fromTxId, boolean inProgressOk) throws IOException {
@@ -1477,4 +1532,5 @@ public class FSEditLog implements LogsPurgeable {
                                          + uri, e);
                                          + 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
    * @return true if the image needs to be saved or false otherwise
    */
    */
   boolean recoverTransitionRead(StartupOption startOpt, FSNamesystem target,
   boolean recoverTransitionRead(StartupOption startOpt, FSNamesystem target,
-      MetaRecoveryContext recovery) throws IOException {
+      MetaRecoveryContext recovery)
+      throws IOException {
     assert startOpt != StartupOption.FORMAT : 
     assert startOpt != StartupOption.FORMAT : 
       "NameNode formatting should be performed before reading the image";
       "NameNode formatting should be performed before reading the image";
     
     
@@ -252,8 +253,8 @@ public class FSImage implements Closeable {
       doImportCheckpoint(target);
       doImportCheckpoint(target);
       return false; // import checkpoint saved image already
       return false; // import checkpoint saved image already
     case ROLLBACK:
     case ROLLBACK:
-      doRollback();
-      break;
+      throw new AssertionError("Rollback is now a standalone command, " +
+          "NameNode should not be starting with this option.");
     case REGULAR:
     case REGULAR:
     default:
     default:
       // just load the image
       // just load the image
@@ -272,17 +273,15 @@ public class FSImage implements Closeable {
   private boolean recoverStorageDirs(StartupOption startOpt,
   private boolean recoverStorageDirs(StartupOption startOpt,
       Map<StorageDirectory, StorageState> dataDirStates) throws IOException {
       Map<StorageDirectory, StorageState> dataDirStates) throws IOException {
     boolean isFormatted = false;
     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 = 
     for (Iterator<StorageDirectory> it = 
                       storage.dirIterator(); it.hasNext();) {
                       storage.dirIterator(); it.hasNext();) {
       StorageDirectory sd = it.next();
       StorageDirectory sd = it.next();
       StorageState curState;
       StorageState curState;
       try {
       try {
         curState = sd.analyzeStorage(startOpt, storage);
         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
         // sd is locked but not opened
         switch(curState) {
         switch(curState) {
         case NON_EXISTENT:
         case NON_EXISTENT:
@@ -294,7 +293,7 @@ public class FSImage implements Closeable {
         case NORMAL:
         case NORMAL:
           break;
           break;
         default:  // recovery is possible
         default:  // recovery is possible
-          sd.doRecover(curState);      
+          sd.doRecover(curState);
         }
         }
         if (curState != StorageState.NOT_FORMATTED 
         if (curState != StorageState.NOT_FORMATTED 
             && startOpt != StartupOption.ROLLBACK) {
             && startOpt != StartupOption.ROLLBACK) {
@@ -315,10 +314,10 @@ public class FSImage implements Closeable {
     return isFormatted;
     return isFormatted;
   }
   }
 
 
-  private void doUpgrade(FSNamesystem target) throws IOException {
+  void doUpgrade(FSNamesystem target) throws IOException {
     // Upgrade is allowed only if there are 
     // 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();
       StorageDirectory sd = it.next();
       if (sd.getPreviousDir().exists())
       if (sd.getPreviousDir().exists())
         throw new InconsistentFSStateException(sd.getRoot(),
         throw new InconsistentFSStateException(sd.getRoot(),
@@ -327,9 +326,10 @@ public class FSImage implements Closeable {
     }
     }
 
 
     // load the latest image
     // load the latest image
-    this.loadFSImage(target, null, null);
 
 
     // Do upgrade for each directory
     // Do upgrade for each directory
+    this.loadFSImage(target, StartupOption.UPGRADE, null);
+    
     long oldCTime = storage.getCTime();
     long oldCTime = storage.getCTime();
     storage.cTime = now();  // generate new cTime for the state
     storage.cTime = now();  // generate new cTime for the state
     int oldLV = storage.getLayoutVersion();
     int oldLV = storage.getLayoutVersion();
@@ -337,28 +337,17 @@ public class FSImage implements Closeable {
     
     
     List<StorageDirectory> errorSDs =
     List<StorageDirectory> errorSDs =
       Collections.synchronizedList(new ArrayList<StorageDirectory>());
       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();
       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 {
       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) {
       } catch (Exception e) {
         LOG.error("Failed to move aside pre-upgrade storage " +
         LOG.error("Failed to move aside pre-upgrade storage " +
             "in image directory " + sd.getRoot(), e);
             "in image directory " + sd.getRoot(), e);
@@ -366,41 +355,38 @@ public class FSImage implements Closeable {
         continue;
         continue;
       }
       }
     }
     }
+    if (target.isHaEnabled()) {
+      editLog.doPreUpgradeOfSharedLog();
+    }
     storage.reportErrorsOnDirectories(errorSDs);
     storage.reportErrorsOnDirectories(errorSDs);
     errorSDs.clear();
     errorSDs.clear();
 
 
     saveFSImageInAllDirs(target, editLog.getLastWrittenTxId());
     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();
       StorageDirectory sd = it.next();
       try {
       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) {
       } catch (IOException ioe) {
-        LOG.error("Unable to rename temp to previous for " + sd.getRoot(), ioe);
         errorSDs.add(sd);
         errorSDs.add(sd);
         continue;
         continue;
       }
       }
-      LOG.info("Upgrade of " + sd.getRoot() + " is complete.");
+    }
+    if (target.isHaEnabled()) {
+      editLog.doUpgradeOfSharedLog();
     }
     }
     storage.reportErrorsOnDirectories(errorSDs);
     storage.reportErrorsOnDirectories(errorSDs);
-
+    
     isUpgradeFinalized = false;
     isUpgradeFinalized = false;
     if (!storage.getRemovedStorageDirs().isEmpty()) {
     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 "
       throw new IOException("Upgrade failed in "
           + storage.getRemovedStorageDirs().size()
           + storage.getRemovedStorageDirs().size()
           + " storage directory(ies), previously logged.");
           + " storage directory(ies), previously logged.");
     }
     }
   }
   }
 
 
-  private void doRollback() throws IOException {
+  void doRollback(FSNamesystem fsns) throws IOException {
     // Rollback is allowed only if there is 
     // Rollback is allowed only if there is 
     // a previous fs states in at least one of the storage directories.
     // a previous fs states in at least one of the storage directories.
     // Directories that don't have previous state do not rollback
     // Directories that don't have previous state do not rollback
@@ -408,85 +394,46 @@ public class FSImage implements Closeable {
     FSImage prevState = new FSImage(conf);
     FSImage prevState = new FSImage(conf);
     try {
     try {
       prevState.getStorage().layoutVersion = HdfsConstants.LAYOUT_VERSION;
       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();
         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;
           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;
         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)
       if (!canRollback)
         throw new IOException("Cannot rollback. None of the storage "
         throw new IOException("Cannot rollback. None of the storage "
             + "directories contain previous fs state.");
             + "directories contain previous fs state.");
-
+  
       // Now that we know all directories are going to be consistent
       // Now that we know all directories are going to be consistent
       // Do rollback for each directory containing previous state
       // 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();
         StorageDirectory sd = it.next();
-        File prevDir = sd.getPreviousDir();
-        if (!prevDir.exists())
-          continue;
-
         LOG.info("Rolling back storage directory " + sd.getRoot()
         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;
       isUpgradeFinalized = true;
     } finally {
     } finally {
       prevState.close();
       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.
    * Load image from a checkpoint directory and save it into the current one.
    * @param target the NameSystem to import into
    * @param target the NameSystem to import into
@@ -521,7 +468,7 @@ public class FSImage implements Closeable {
     // return back the real image
     // return back the real image
     realImage.getStorage().setStorageInfo(ckptImage.getStorage());
     realImage.getStorage().setStorageInfo(ckptImage.getStorage());
     realImage.getEditLog().setNextTxId(ckptImage.getEditLog().getLastWrittenTxId()+1);
     realImage.getEditLog().setNextTxId(ckptImage.getEditLog().getLastWrittenTxId()+1);
-    realImage.initEditLog();
+    realImage.initEditLog(StartupOption.IMPORT);
 
 
     target.dir.fsImage = realImage;
     target.dir.fsImage = realImage;
     realImage.getStorage().setBlockPoolID(ckptImage.getBlockPoolID());
     realImage.getStorage().setBlockPoolID(ckptImage.getBlockPoolID());
@@ -530,12 +477,23 @@ public class FSImage implements Closeable {
     saveNamespace(target);
     saveNamespace(target);
     getStorage().writeAll();
     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();
       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() {
   boolean isUpgradeFinalized() {
@@ -601,7 +559,7 @@ public class FSImage implements Closeable {
 
 
     Iterable<EditLogInputStream> editStreams = null;
     Iterable<EditLogInputStream> editStreams = null;
 
 
-    initEditLog();
+    initEditLog(startOpt);
 
 
     if (LayoutVersion.supports(Feature.TXID_BASED_LAYOUT, 
     if (LayoutVersion.supports(Feature.TXID_BASED_LAYOUT, 
                                getLayoutVersion())) {
                                getLayoutVersion())) {
@@ -683,14 +641,30 @@ public class FSImage implements Closeable {
     }
     }
   }
   }
 
 
-  public void initEditLog() {
+  public void initEditLog(StartupOption startOpt) throws IOException {
     Preconditions.checkState(getNamespaceID() != 0,
     Preconditions.checkState(getNamespaceID() != 0,
         "Must know namespace ID before initting edit log");
         "Must know namespace ID before initting edit log");
     String nameserviceId = DFSUtil.getNamenodeNameServiceId(conf);
     String nameserviceId = DFSUtil.getNamenodeNameServiceId(conf);
     if (!HAUtil.isHAEnabled(conf, nameserviceId)) {
     if (!HAUtil.isHAEnabled(conf, nameserviceId)) {
+      // If this NN is not HA
       editLog.initJournalsForWrite();
       editLog.initJournalsForWrite();
       editLog.recoverUnclosedStreams();
       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 {
     } else {
+      // This NN is HA and we're not doing an upgrade.
       editLog.initSharedJournalsForRead();
       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_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_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_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_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_KEY;
 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;
 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.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 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.StringUtils;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.VersionInfo;
 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 org.mortbay.util.ajax.JSON;
 
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.annotations.VisibleForTesting;
@@ -550,6 +556,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     return leaseManager;
     return leaseManager;
   }
   }
   
   
+  boolean isHaEnabled() {
+    return haEnabled;
+  }
+  
   /**
   /**
    * Check the supplied configuration for correctness.
    * Check the supplied configuration for correctness.
    * @param conf Supplies the configuration to validate.
    * @param conf Supplies the configuration to validate.
@@ -661,6 +671,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    */
    */
   FSNamesystem(Configuration conf, FSImage fsImage, boolean ignoreRetryCache)
   FSNamesystem(Configuration conf, FSImage fsImage, boolean ignoreRetryCache)
       throws IOException {
       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);
     boolean fair = conf.getBoolean("dfs.namenode.fslock.fair", true);
     LOG.info("fsLock is fair:" + fair);
     LOG.info("fsLock is fair:" + fair);
     fsLock = new FSNamesystemLock(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
       // 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
       // we shouldn't do it when coming up in standby state
-      if (!haEnabled) {
+      if (!haEnabled || (haEnabled && startOpt == StartupOption.UPGRADE)) {
         fsImage.openEditLogForWrite();
         fsImage.openEditLogForWrite();
       }
       }
       success = true;
       success = true;
@@ -1000,6 +1015,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
 
         dir.fsImage.editLog.openForWrite();
         dir.fsImage.editLog.openForWrite();
       }
       }
+      
       if (haEnabled) {
       if (haEnabled) {
         // Renew all of the leases before becoming active.
         // Renew all of the leases before becoming active.
         // This is because, while we were in standby mode,
         // 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.
    * Initialize replication queues.
    */
    */
@@ -1040,9 +1061,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    *         middle of the {@link #startActiveServices()}
    *         middle of the {@link #startActiveServices()}
    */
    */
   public boolean inTransitionToActive() {
   public boolean inTransitionToActive() {
-    return haEnabled && haContext != null
-        && haContext.getState().getServiceState() == HAServiceState.ACTIVE
-        && startingActiveService;
+    return haEnabled && inActiveState() && startingActiveService;
   }
   }
 
 
   private boolean shouldUseDelegationTokens() {
   private boolean shouldUseDelegationTokens() {
@@ -4519,11 +4538,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     
     
   void finalizeUpgrade() throws IOException {
   void finalizeUpgrade() throws IOException {
     checkSuperuserPrivilege();
     checkSuperuserPrivilege();
-    checkOperation(OperationCategory.WRITE);
+    checkOperation(OperationCategory.UNCHECKED);
     writeLock();
     writeLock();
     try {
     try {
-      checkOperation(OperationCategory.WRITE);
-      getFSImage().finalizeUpgrade();
+      checkOperation(OperationCategory.UNCHECKED);
+      getFSImage().finalizeUpgrade(this.isHaEnabled() && inActiveState());
     } finally {
     } finally {
       writeUnlock();
       writeUnlock();
     }
     }
@@ -6783,6 +6802,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   
   
   /** Allow snapshot on a directroy. */
   /** Allow snapshot on a directroy. */
   void allowSnapshot(String path) throws SafeModeException, IOException {
   void allowSnapshot(String path) throws SafeModeException, IOException {
+    checkOperation(OperationCategory.WRITE);
     writeLock();
     writeLock();
     try {
     try {
       checkOperation(OperationCategory.WRITE);
       checkOperation(OperationCategory.WRITE);
@@ -6808,6 +6828,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   
   
   /** Disallow snapshot on a directory. */
   /** Disallow snapshot on a directory. */
   void disallowSnapshot(String path) throws SafeModeException, IOException {
   void disallowSnapshot(String path) throws SafeModeException, IOException {
+    checkOperation(OperationCategory.WRITE);
     writeLock();
     writeLock();
     try {
     try {
       checkOperation(OperationCategory.WRITE);
       checkOperation(OperationCategory.WRITE);
@@ -6931,6 +6952,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
   public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
       throws IOException {
       throws IOException {
     SnapshottableDirectoryStatus[] status = null;
     SnapshottableDirectoryStatus[] status = null;
+    checkOperation(OperationCategory.READ);
     final FSPermissionChecker checker = getPermissionChecker();
     final FSPermissionChecker checker = getPermissionChecker();
     readLock();
     readLock();
     try {
     try {
@@ -6964,6 +6986,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   SnapshotDiffReport getSnapshotDiffReport(String path,
   SnapshotDiffReport getSnapshotDiffReport(String path,
       String fromSnapshot, String toSnapshot) throws IOException {
       String fromSnapshot, String toSnapshot) throws IOException {
     SnapshotDiffInfo diffs = null;
     SnapshotDiffInfo diffs = null;
+    checkOperation(OperationCategory.READ);
     final FSPermissionChecker pc = getPermissionChecker();
     final FSPermissionChecker pc = getPermissionChecker();
     readLock();
     readLock();
     try {
     try {
@@ -7445,5 +7468,27 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       auditLog.info(message);
       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.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 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.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.common.StorageErrorReporter;
 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.NNStorageRetentionManager.StoragePurger;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogLoader.EditLogValidation;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogLoader.EditLogValidation;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
-
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Joiner;
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Preconditions;
@@ -489,4 +490,49 @@ public class FileJournalManager implements JournalManager {
                            isInProgress(), hasCorruptHeader);
                            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.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 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.Storage.FormatConfirmable;
+import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 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.
    * Recover segments which have not been finalized.
    */
    */
   void recoverUnfinalizedSegments() throws IOException;
   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.
    * Close the journal manager, freeing any resources it may hold.
@@ -84,4 +134,5 @@ public interface JournalManager extends Closeable, FormatConfirmable,
       super(reason);
       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.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 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.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
@@ -77,11 +79,14 @@ public class JournalSet implements JournalManager {
     private final JournalManager journal;
     private final JournalManager journal;
     private boolean disabled = false;
     private boolean disabled = false;
     private EditLogOutputStream stream;
     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.journal = manager;
       this.required = required;
       this.required = required;
+      this.shared = shared;
     }
     }
 
 
     public void startLogSegment(long txId) throws IOException {
     public void startLogSegment(long txId) throws IOException {
@@ -163,6 +168,10 @@ public class JournalSet implements JournalManager {
     public boolean isRequired() {
     public boolean isRequired() {
       return required;
       return required;
     }
     }
+    
+    public boolean isShared() {
+      return shared;
+    }
   }
   }
  
  
   // COW implementation is necessary since some users (eg the web ui) call
   // COW implementation is necessary since some users (eg the web ui) call
@@ -178,7 +187,7 @@ public class JournalSet implements JournalManager {
   
   
   @Override
   @Override
   public void format(NamespaceInfo nsInfo) throws IOException {
   public void format(NamespaceInfo nsInfo) throws IOException {
-    // The iteration is done by FSEditLog itself
+    // The operation is done by FSEditLog itself
     throw new UnsupportedOperationException();
     throw new UnsupportedOperationException();
   }
   }
 
 
@@ -537,9 +546,13 @@ public class JournalSet implements JournalManager {
     }
     }
     return jList;
     return jList;
   }
   }
-
+  
   void add(JournalManager j, boolean required) {
   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);
     journals.add(jas);
   }
   }
   
   
@@ -655,4 +668,40 @@ public class JournalSet implements JournalManager {
     }
     }
     return buf.toString();
     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) {
       if(dirName.getScheme().compareTo("file") == 0) {
         this.addStorageDir(new StorageDirectory(new File(dirName.getPath()),
         this.addStorageDir(new StorageDirectory(new File(dirName.getPath()),
             dirType,
             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://
       // URI is of type file://
       if(dirName.getScheme().compareTo("file") == 0)
       if(dirName.getScheme().compareTo("file") == 0)
         this.addStorageDir(new StorageDirectory(new File(dirName.getPath()),
         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();
     StringBuilder layoutVersions = new StringBuilder();
 
 
     // First determine what range of layout versions we're going to inspect
     // 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();) {
          it.hasNext();) {
       StorageDirectory sd = it.next();
       StorageDirectory sd = it.next();
       if (!sd.getVersionFile().exists()) {
       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 nsId = getNameServiceId(conf);
     String namenodeId = HAUtil.getNameNodeId(conf, nsId);
     String namenodeId = HAUtil.getNameNodeId(conf, nsId);
     this.haEnabled = HAUtil.isHAEnabled(conf, nsId);
     this.haEnabled = HAUtil.isHAEnabled(conf, nsId);
-    state = createHAState();
+    state = createHAState(getStartupOption(conf));
     this.allowStaleStandbyReads = HAUtil.shouldAllowStandbyReads(conf);
     this.allowStaleStandbyReads = HAUtil.shouldAllowStandbyReads(conf);
     this.haContext = createHAContext();
     this.haContext = createHAContext();
     try {
     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() {
   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 nsId = DFSUtil.getNamenodeNameServiceId(conf);
     String namenodeId = HAUtil.getNameNodeId(conf, nsId);
     String namenodeId = HAUtil.getNameNodeId(conf, nsId);
     initializeGenericKeys(conf, nsId, namenodeId);
     initializeGenericKeys(conf, nsId, namenodeId);
 
 
     FSNamesystem nsys = new FSNamesystem(conf, new FSImage(conf));
     FSNamesystem nsys = new FSNamesystem(conf, new FSImage(conf));
     System.err.print(
     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 (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;
         return true;
       }
       }
     }
     }
-    nsys.dir.fsImage.finalizeUpgrade();
+    nsys.dir.fsImage.doRollback(nsys);
     return false;
     return false;
   }
   }
 
 
@@ -1210,14 +1216,6 @@ public class NameNode implements NameNodeStatusMXBean {
       return null;
       return null;
     }
     }
     setStartupOption(conf, startOpt);
     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) {
     switch (startOpt) {
       case FORMAT: {
       case FORMAT: {
@@ -1233,10 +1231,17 @@ public class NameNode implements NameNodeStatusMXBean {
         return null;
         return null;
       }
       }
       case FINALIZE: {
       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
         return null; // avoid javac warning
       }
       }
+      case ROLLBACK: {
+        boolean aborted = doRollback(conf, true);
+        terminate(aborted ? 1 : 0);
+        return null; // avoid warning
+      }
       case BOOTSTRAPSTANDBY: {
       case BOOTSTRAPSTANDBY: {
         String toolArgs[] = Arrays.copyOfRange(argv, 1, argv.length);
         String toolArgs[] = Arrays.copyOfRange(argv, 1, argv.length);
         int rc = BootstrapStandby.run(toolArgs, conf);
         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.NameNodeProxies;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.Storage;
 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.EditLogInputStream;
 import org.apache.hadoop.hdfs.server.namenode.FSImage;
 import org.apache.hadoop.hdfs.server.namenode.FSImage;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
@@ -192,7 +193,7 @@ public class BootstrapStandby implements Tool, Configurable {
     FSImage image = new FSImage(conf);
     FSImage image = new FSImage(conf);
     try {
     try {
       image.getStorage().setStorageInfo(storage);
       image.getStorage().setStorageInfo(storage);
-      image.initEditLog();
+      image.initEditLog(StartupOption.REGULAR);
       assert image.getEditLog().isOpenForRead() :
       assert image.getEditLog().isOpenForRead() :
         "Expected edit log to be open for read";
         "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.File;
 import java.io.IOException;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
+import java.net.URI;
 import java.net.URL;
 import java.net.URL;
 import java.security.PrivilegedExceptionAction;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 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.HdfsConfiguration;
 import org.apache.hadoop.hdfs.NameNodeProxies;
 import org.apache.hadoop.hdfs.NameNodeProxies;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 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.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
@@ -830,7 +832,24 @@ public class DFSAdmin extends FsShell {
    */
    */
   public int finalizeUpgrade() throws IOException {
   public int finalizeUpgrade() throws IOException {
     DistributedFileSystem dfs = getDFS();
     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;
     return 0;
   }
   }

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

@@ -133,6 +133,72 @@ message IsFormattedResponseProto {
   required bool isFormatted = 1;
   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()
  * getJournalState()
  */
  */
@@ -236,6 +302,18 @@ message AcceptRecoveryResponseProto {
 service QJournalProtocolService {
 service QJournalProtocolService {
   rpc isFormatted(IsFormattedRequestProto) returns (IsFormattedResponseProto);
   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 getJournalState(GetJournalStateRequestProto) returns (GetJournalStateResponseProto);
 
 
   rpc newEpoch(NewEpochRequestProto) returns (NewEpochResponseProto);
   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>
 <title>Namenode information</title>
 </head>
 </head>
 <body>
 <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="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>
-<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>
+
 <div class="row">
 <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>
 </div>
 
 
 <script type="text/x-dust-template" id="tmpl-dfshealth">
 <script type="text/x-dust-template" id="tmpl-dfshealth">
-<div class="page-header">
-  {#nnstat}
-  <h1>NameNode '{HostAndPort}' ({State})</h1>
-  {/nnstat}
-</div>
 
 
 {#nn}
 {#nn}
 {@if cond="{DistinctVersionCount} > 1"}
 {@if cond="{DistinctVersionCount} > 1"}
@@ -71,238 +99,193 @@
 {/if}
 {/if}
 {/nn}
 {/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>
 	<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>
 	<tr><th>Journal Manager</th><th>State</th></tr>
-      </thead>
-      <tbody>
+  </thead>
+  <tbody>
 	{#nn.NameJournalStatus}
 	{#nn.NameJournalStatus}
 	<tr><td>{manager}</td><td>{stream}</td></tr>
 	<tr><td>{manager}</td><td>{stream}</td></tr>
 	{/nn.NameJournalStatus}
 	{/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>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>
 	</tr>
-      </thead>
-      <tbody>
+  </thead>
+  <tbody>
 	{#phases}
 	{#phases}
 	<tr class="phase">
 	<tr class="phase">
 	  <td class="startupdesc">{desc} {file} {size|fmt_bytes}</td>
 	  <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>
 	</tr>
 	{#steps root_file=file}
 	{#steps root_file=file}
 	<tr class="step">
 	<tr class="step">
 	  <td class="startupdesc">{stepDesc} {stepFile} {stepSize|fmt_bytes} ({count}/{total})</td>
 	  <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>
 	  <td></td>
 	</tr>
 	</tr>
 	{/steps}
 	{/steps}
 	{/phases}
 	{/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>
 
 
 <script type="text/javascript" src="/static/jquery-1.10.2.min.js">
 <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 () {
 (function () {
   "use strict";
   "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) {
       'helper_fs_max_objects': function (chunk, ctx, bodies, params) {
         var o = ctx.current();
         var o = ctx.current();
         if (o.MaxObjects > 0) {
         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) {
       function rename_property(o, s, d) {
         if (o[s] !== undefined) {
         if (o[s] !== undefined) {
           o[d] = o[s];
           o[d] = o[s];
@@ -86,36 +114,66 @@
       });
       });
       return r;
       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 {
 .panel-success > .panel-heading {
   color: #fff !important;
   color: #fff !important;
   background-color: #5FA33E !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
   Even if automatic failover is configured, you may initiate a manual failover
   using the same <<<hdfs haadmin>>> command. It will perform a coordinated
   using the same <<<hdfs haadmin>>> command. It will perform a coordinated
   failover.
   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 enableManagedDfsDirsRedundancy = true;
     private boolean manageDataDfsDirs = true;
     private boolean manageDataDfsDirs = true;
     private StartupOption option = null;
     private StartupOption option = null;
+    private StartupOption dnOption = null;
     private String[] racks = null; 
     private String[] racks = null; 
     private String [] hosts = null;
     private String [] hosts = null;
     private long [] simulatedCapacities = null;
     private long [] simulatedCapacities = null;
@@ -240,6 +241,14 @@ public class MiniDFSCluster {
       this.option = val;
       this.option = val;
       return this;
       return this;
     }
     }
+    
+    /**
+     * Default: null
+     */
+    public Builder dnStartupOption(StartupOption val) {
+      this.dnOption = val;
+      return this;
+    }
 
 
     /**
     /**
      * Default: null
      * Default: null
@@ -356,6 +365,7 @@ public class MiniDFSCluster {
                        builder.enableManagedDfsDirsRedundancy,
                        builder.enableManagedDfsDirsRedundancy,
                        builder.manageDataDfsDirs,
                        builder.manageDataDfsDirs,
                        builder.option,
                        builder.option,
+                       builder.dnOption,
                        builder.racks,
                        builder.racks,
                        builder.hosts,
                        builder.hosts,
                        builder.simulatedCapacities,
                        builder.simulatedCapacities,
@@ -405,18 +415,24 @@ public class MiniDFSCluster {
   /**
   /**
    * Stores the information related to a namenode in the cluster
    * Stores the information related to a namenode in the cluster
    */
    */
-  static class NameNodeInfo {
+  public static class NameNodeInfo {
     final NameNode nameNode;
     final NameNode nameNode;
     final Configuration conf;
     final Configuration conf;
     final String nameserviceId;
     final String nameserviceId;
     final String nnId;
     final String nnId;
+    StartupOption startOpt;
     NameNodeInfo(NameNode nn, String nameserviceId, String nnId,
     NameNodeInfo(NameNode nn, String nameserviceId, String nnId,
-        Configuration conf) {
+        StartupOption startOpt, Configuration conf) {
       this.nameNode = nn;
       this.nameNode = nn;
       this.nameserviceId = nameserviceId;
       this.nameserviceId = nameserviceId;
       this.nnId = nnId;
       this.nnId = nnId;
+      this.startOpt = startOpt;
       this.conf = conf;
       this.conf = conf;
     }
     }
+    
+    public void setStartOpt(StartupOption startOpt) {
+      this.startOpt = startOpt;
+    }
   }
   }
   
   
   /**
   /**
@@ -602,8 +618,8 @@ public class MiniDFSCluster {
                         long[] simulatedCapacities) throws IOException {
                         long[] simulatedCapacities) throws IOException {
     this.nameNodes = new NameNodeInfo[1]; // Single namenode in the cluster
     this.nameNodes = new NameNodeInfo[1]; // Single namenode in the cluster
     initMiniDFSCluster(conf, numDataNodes, StorageType.DEFAULT, format,
     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,
         simulatedCapacities, null, true, false,
         MiniDFSNNTopology.simpleSingleNN(nameNodePort, 0), true, false, false);
         MiniDFSNNTopology.simpleSingleNN(nameNodePort, 0), true, false, false);
   }
   }
@@ -612,7 +628,8 @@ public class MiniDFSCluster {
       Configuration conf,
       Configuration conf,
       int numDataNodes, StorageType storageType, boolean format, boolean manageNameDfsDirs,
       int numDataNodes, StorageType storageType, boolean format, boolean manageNameDfsDirs,
       boolean manageNameDfsSharedDirs, boolean enableManagedDfsDirsRedundancy,
       boolean manageNameDfsSharedDirs, boolean enableManagedDfsDirsRedundancy,
-      boolean manageDataDfsDirs, StartupOption operation, String[] racks,
+      boolean manageDataDfsDirs, StartupOption startOpt,
+      StartupOption dnStartOpt, String[] racks,
       String[] hosts, long[] simulatedCapacities, String clusterId,
       String[] hosts, long[] simulatedCapacities, String clusterId,
       boolean waitSafeMode, boolean setupHostsFile,
       boolean waitSafeMode, boolean setupHostsFile,
       MiniDFSNNTopology nnTopology, boolean checkExitOnShutdown,
       MiniDFSNNTopology nnTopology, boolean checkExitOnShutdown,
@@ -661,7 +678,7 @@ public class MiniDFSCluster {
       createNameNodesAndSetConf(
       createNameNodesAndSetConf(
           nnTopology, manageNameDfsDirs, manageNameDfsSharedDirs,
           nnTopology, manageNameDfsDirs, manageNameDfsSharedDirs,
           enableManagedDfsDirsRedundancy,
           enableManagedDfsDirsRedundancy,
-          format, operation, clusterId, conf);
+          format, startOpt, clusterId, conf);
     } catch (IOException ioe) {
     } catch (IOException ioe) {
       LOG.error("IOE creating namenodes. Permissions dump:\n" +
       LOG.error("IOE creating namenodes. Permissions dump:\n" +
           createPermissionsDiagnosisString(data_dir));
           createPermissionsDiagnosisString(data_dir));
@@ -674,13 +691,15 @@ public class MiniDFSCluster {
       }
       }
     }
     }
     
     
-    if (operation == StartupOption.RECOVER) {
+    if (startOpt == StartupOption.RECOVER) {
       return;
       return;
     }
     }
 
 
     // Start the DataNodes
     // 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();
     waitClusterUp();
     //make sure ProxyUsers uses the latest conf
     //make sure ProxyUsers uses the latest conf
     ProxyUsers.refreshSuperUserGroupsConfiguration(conf);
     ProxyUsers.refreshSuperUserGroupsConfiguration(conf);
@@ -758,6 +777,8 @@ public class MiniDFSCluster {
         if (manageNameDfsSharedDirs) {
         if (manageNameDfsSharedDirs) {
           URI sharedEditsUri = getSharedEditsDir(nnCounter, nnCounter+nnIds.size()-1); 
           URI sharedEditsUri = getSharedEditsDir(nnCounter, nnCounter+nnIds.size()-1); 
           conf.set(DFS_NAMENODE_SHARED_EDITS_DIR_KEY, sharedEditsUri.toString());
           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);
     URI srcDir = Lists.newArrayList(srcDirs).get(0);
     FileSystem dstFS = FileSystem.getLocal(dstConf).getRaw();
     FileSystem dstFS = FileSystem.getLocal(dstConf).getRaw();
     for (URI dstDir : dstDirs) {
     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);
       File dstDirF = new File(dstDir);
       if (dstDirF.exists()) {
       if (dstDirF.exists()) {
         if (!FileUtil.fullyDelete(dstDirF)) {
         if (!FileUtil.fullyDelete(dstDirF)) {
@@ -891,6 +913,14 @@ public class MiniDFSCluster {
     conf.set(key, "127.0.0.1:" + nnConf.getIpcPort());
     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,
   private void createNameNode(int nnIndex, Configuration conf,
       int numDataNodes, boolean format, StartupOption operation,
       int numDataNodes, boolean format, StartupOption operation,
       String clusterId, String nameserviceId,
       String clusterId, String nameserviceId,
@@ -905,10 +935,7 @@ public class MiniDFSCluster {
     }
     }
     
     
     // Start the NameNode
     // 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);
     NameNode nn =  NameNode.createNameNode(args, conf);
     if (operation == StartupOption.RECOVER) {
     if (operation == StartupOption.RECOVER) {
       return;
       return;
@@ -930,7 +957,7 @@ public class MiniDFSCluster {
     DFSUtil.setGenericConf(conf, nameserviceId, nnId,
     DFSUtil.setGenericConf(conf, nameserviceId, nnId,
         DFS_NAMENODE_HTTP_ADDRESS_KEY);
         DFS_NAMENODE_HTTP_ADDRESS_KEY);
     nameNodes[nnIndex] = new NameNodeInfo(nn, nameserviceId, nnId,
     nameNodes[nnIndex] = new NameNodeInfo(nn, nameserviceId, nnId,
-        new Configuration(conf));
+        operation, new Configuration(conf));
   }
   }
 
 
   /**
   /**
@@ -1498,7 +1525,7 @@ public class MiniDFSCluster {
       nn.stop();
       nn.stop();
       nn.join();
       nn.join();
       Configuration conf = nameNodes[nnIndex].conf;
       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... args) throws IOException {
     String nameserviceId = nameNodes[nnIndex].nameserviceId;
     String nameserviceId = nameNodes[nnIndex].nameserviceId;
     String nnId = nameNodes[nnIndex].nnId;
     String nnId = nameNodes[nnIndex].nnId;
+    StartupOption startOpt = nameNodes[nnIndex].startOpt;
     Configuration conf = nameNodes[nnIndex].conf;
     Configuration conf = nameNodes[nnIndex].conf;
     shutdownNameNode(nnIndex);
     shutdownNameNode(nnIndex);
+    if (args.length != 0) {
+      startOpt = null;
+    } else {
+      args = createArgs(startOpt);
+    }
     NameNode nn = NameNode.createNameNode(args, conf);
     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) {
     if (waitActive) {
       waitClusterUp();
       waitClusterUp();
       LOG.info("Restarted the namenode");
       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.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
 import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.junit.After;
 import org.junit.After;
 import org.junit.Test;
 import org.junit.Test;
@@ -97,10 +98,10 @@ public class TestDFSRollback {
    * Attempts to start a NameNode with the given operation.  Starting
    * Attempts to start a NameNode with the given operation.  Starting
    * the NameNode should throw an exception.
    * the NameNode should throw an exception.
    */
    */
-  void startNameNodeShouldFail(StartupOption operation, String searchString) {
+  void startNameNodeShouldFail(String searchString) {
     try {
     try {
+      NameNode.doRollback(conf, false);
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
-                                                .startupOption(operation)
                                                 .format(false)
                                                 .format(false)
                                                 .manageDataDfsDirs(false)
                                                 .manageDataDfsDirs(false)
                                                 .manageNameDfsDirs(false)
                                                 .manageNameDfsDirs(false)
@@ -149,24 +150,19 @@ public class TestDFSRollback {
       log("Normal NameNode rollback", numDirs);
       log("Normal NameNode rollback", numDirs);
       UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "current");
       UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "current");
       UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "previous");
       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);
       checkResult(NAME_NODE, nameNodeDirs);
-      cluster.shutdown();
       UpgradeUtilities.createEmptyDirs(nameNodeDirs);
       UpgradeUtilities.createEmptyDirs(nameNodeDirs);
       
       
       log("Normal DataNode rollback", numDirs);
       log("Normal DataNode rollback", numDirs);
       UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "current");
       UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "current");
       UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "previous");
       UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "previous");
+      NameNode.doRollback(conf, false);
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
                                                 .format(false)
                                                 .format(false)
                                                 .manageDataDfsDirs(false)
                                                 .manageDataDfsDirs(false)
                                                 .manageNameDfsDirs(false)
                                                 .manageNameDfsDirs(false)
-                                                .startupOption(StartupOption.ROLLBACK)
+                                                .dnStartupOption(StartupOption.ROLLBACK)
                                                 .build();
                                                 .build();
       UpgradeUtilities.createDataNodeStorageDirs(dataNodeDirs, "current");
       UpgradeUtilities.createDataNodeStorageDirs(dataNodeDirs, "current");
       UpgradeUtilities.createDataNodeStorageDirs(dataNodeDirs, "previous");
       UpgradeUtilities.createDataNodeStorageDirs(dataNodeDirs, "previous");
@@ -179,11 +175,12 @@ public class TestDFSRollback {
       log("Normal BlockPool rollback", numDirs);
       log("Normal BlockPool rollback", numDirs);
       UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "current");
       UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "current");
       UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "previous");
       UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "previous");
+      NameNode.doRollback(conf, false);
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
                                                 .format(false)
                                                 .format(false)
                                                 .manageDataDfsDirs(false)
                                                 .manageDataDfsDirs(false)
                                                 .manageNameDfsDirs(false)
                                                 .manageNameDfsDirs(false)
-                                                .startupOption(StartupOption.ROLLBACK)
+                                                .dnStartupOption(StartupOption.ROLLBACK)
                                                 .build();
                                                 .build();
       UpgradeUtilities.createDataNodeStorageDirs(dataNodeDirs, "current");
       UpgradeUtilities.createDataNodeStorageDirs(dataNodeDirs, "current");
       UpgradeUtilities.createBlockPoolStorageDirs(dataNodeDirs, "current",
       UpgradeUtilities.createBlockPoolStorageDirs(dataNodeDirs, "current",
@@ -217,10 +214,10 @@ public class TestDFSRollback {
       cluster.shutdown();
       cluster.shutdown();
       UpgradeUtilities.createEmptyDirs(nameNodeDirs);
       UpgradeUtilities.createEmptyDirs(nameNodeDirs);
       UpgradeUtilities.createEmptyDirs(dataNodeDirs);
       UpgradeUtilities.createEmptyDirs(dataNodeDirs);
-
+      
       log("NameNode rollback without existing previous dir", numDirs);
       log("NameNode rollback without existing previous dir", numDirs);
       UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "current");
       UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "current");
-      startNameNodeShouldFail(StartupOption.ROLLBACK,
+      startNameNodeShouldFail(
           "None of the storage directories contain previous fs state");
           "None of the storage directories contain previous fs state");
       UpgradeUtilities.createEmptyDirs(nameNodeDirs);
       UpgradeUtilities.createEmptyDirs(nameNodeDirs);
       
       
@@ -237,15 +234,16 @@ public class TestDFSRollback {
       cluster.shutdown();
       cluster.shutdown();
       UpgradeUtilities.createEmptyDirs(nameNodeDirs);
       UpgradeUtilities.createEmptyDirs(nameNodeDirs);
       UpgradeUtilities.createEmptyDirs(dataNodeDirs);
       UpgradeUtilities.createEmptyDirs(dataNodeDirs);
-
+      
       log("DataNode rollback with future stored layout version in previous", numDirs);
       log("DataNode rollback with future stored layout version in previous", numDirs);
       UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "current");
       UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "current");
       UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "previous");
       UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "previous");
+      NameNode.doRollback(conf, false);
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
                                                 .format(false)
                                                 .format(false)
                                                 .manageDataDfsDirs(false)
                                                 .manageDataDfsDirs(false)
                                                 .manageNameDfsDirs(false)
                                                 .manageNameDfsDirs(false)
-                                                .startupOption(StartupOption.ROLLBACK)
+                                                .dnStartupOption(StartupOption.ROLLBACK)
                                                 .build();
                                                 .build();
       UpgradeUtilities.createDataNodeStorageDirs(dataNodeDirs, "current");
       UpgradeUtilities.createDataNodeStorageDirs(dataNodeDirs, "current");
       baseDirs = UpgradeUtilities.createDataNodeStorageDirs(dataNodeDirs, "previous");
       baseDirs = UpgradeUtilities.createDataNodeStorageDirs(dataNodeDirs, "previous");
@@ -266,11 +264,12 @@ public class TestDFSRollback {
       log("DataNode rollback with newer fsscTime in previous", numDirs);
       log("DataNode rollback with newer fsscTime in previous", numDirs);
       UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "current");
       UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "current");
       UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "previous");
       UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "previous");
+      NameNode.doRollback(conf, false);
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
                                                 .format(false)
                                                 .format(false)
                                                 .manageDataDfsDirs(false)
                                                 .manageDataDfsDirs(false)
                                                 .manageNameDfsDirs(false)
                                                 .manageNameDfsDirs(false)
-                                                .startupOption(StartupOption.ROLLBACK)
+                                                .dnStartupOption(StartupOption.ROLLBACK)
                                                 .build();
                                                 .build();
       
       
       UpgradeUtilities.createDataNodeStorageDirs(dataNodeDirs, "current");
       UpgradeUtilities.createDataNodeStorageDirs(dataNodeDirs, "current");
@@ -287,21 +286,19 @@ public class TestDFSRollback {
       cluster.shutdown();
       cluster.shutdown();
       UpgradeUtilities.createEmptyDirs(nameNodeDirs);
       UpgradeUtilities.createEmptyDirs(nameNodeDirs);
       UpgradeUtilities.createEmptyDirs(dataNodeDirs);
       UpgradeUtilities.createEmptyDirs(dataNodeDirs);
-
+      
       log("NameNode rollback with no edits file", numDirs);
       log("NameNode rollback with no edits file", numDirs);
       UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "current");
       UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "current");
       baseDirs = UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "previous");
       baseDirs = UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "previous");
       deleteMatchingFiles(baseDirs, "edits.*");
       deleteMatchingFiles(baseDirs, "edits.*");
-      startNameNodeShouldFail(StartupOption.ROLLBACK,
-          "Gap in transactions");
+      startNameNodeShouldFail("Gap in transactions");
       UpgradeUtilities.createEmptyDirs(nameNodeDirs);
       UpgradeUtilities.createEmptyDirs(nameNodeDirs);
       
       
       log("NameNode rollback with no image file", numDirs);
       log("NameNode rollback with no image file", numDirs);
       UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "current");
       UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "current");
       baseDirs = UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "previous");
       baseDirs = UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "previous");
       deleteMatchingFiles(baseDirs, "fsimage_.*");
       deleteMatchingFiles(baseDirs, "fsimage_.*");
-      startNameNodeShouldFail(StartupOption.ROLLBACK,
-          "No valid image files found");
+      startNameNodeShouldFail("No valid image files found");
       UpgradeUtilities.createEmptyDirs(nameNodeDirs);
       UpgradeUtilities.createEmptyDirs(nameNodeDirs);
       
       
       log("NameNode rollback with corrupt version file", numDirs);
       log("NameNode rollback with corrupt version file", numDirs);
@@ -313,8 +310,7 @@ public class TestDFSRollback {
             "layoutVersion".getBytes(Charsets.UTF_8),
             "layoutVersion".getBytes(Charsets.UTF_8),
             "xxxxxxxxxxxxx".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);
       UpgradeUtilities.createEmptyDirs(nameNodeDirs);
       
       
@@ -328,8 +324,7 @@ public class TestDFSRollback {
       
       
       UpgradeUtilities.createNameNodeVersionFile(conf, baseDirs,
       UpgradeUtilities.createNameNodeVersionFile(conf, baseDirs,
           storageInfo, UpgradeUtilities.getCurrentBlockPoolID(cluster));
           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);
       UpgradeUtilities.createEmptyDirs(nameNodeDirs);
     } // end numDir loop
     } // 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.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.Shell;
 import org.junit.Assume;
 import org.junit.Assume;
 import org.junit.Before;
 import org.junit.Before;
@@ -764,4 +765,37 @@ public class TestDFSUtil {
     assertEquals(4*24*60*60*1000l, DFSUtil.parseRelativeTime("4d"));
     assertEquals(4*24*60*60*1000l, DFSUtil.parseRelativeTime("4d"));
     assertEquals(999*24*60*60*1000l, DFSUtil.parseRelativeTime("999d"));
     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.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertTrue;
 
 
+import java.io.File;
 import java.io.IOException;
 import java.io.IOException;
+import java.io.RandomAccessFile;
 
 
 import org.apache.hadoop.conf.Configuration;
 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.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
 import org.apache.hadoop.hdfs.protocol.Block;
 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.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 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.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.TestInterDatanodeProtocol;
 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.LeaseManager;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.junit.Test;
 import org.junit.Test;
 
 
 public class TestLeaseRecovery {
 public class TestLeaseRecovery {
@@ -148,4 +156,55 @@ public class TestLeaseRecovery {
       if (cluster != null) {cluster.shutdown();}
       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();
     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) {
   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) {
   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.DFSUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 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.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider;
 import org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider;
 
 
@@ -47,6 +48,7 @@ public class MiniQJMHACluster {
 
 
   public static class Builder {
   public static class Builder {
     private final Configuration conf;
     private final Configuration conf;
+    private StartupOption startOpt = null;
     private final MiniDFSCluster.Builder dfsBuilder;
     private final MiniDFSCluster.Builder dfsBuilder;
     
     
     public Builder(Configuration conf) {
     public Builder(Configuration conf) {
@@ -61,6 +63,10 @@ public class MiniQJMHACluster {
     public MiniQJMHACluster build() throws IOException {
     public MiniQJMHACluster build() throws IOException {
       return new MiniQJMHACluster(this);
       return new MiniQJMHACluster(this);
     }
     }
+
+    public void startupOption(StartupOption startOpt) {
+      this.startOpt = startOpt;
+    }
   }
   }
   
   
   public static MiniDFSNNTopology createDefaultTopology() {
   public static MiniDFSNNTopology createDefaultTopology() {
@@ -95,6 +101,9 @@ public class MiniQJMHACluster {
     Configuration confNN0 = cluster.getConfiguration(0);
     Configuration confNN0 = cluster.getConfiguration(0);
     NameNode.initializeSharedEdits(confNN0, true);
     NameNode.initializeSharedEdits(confNN0, true);
     
     
+    cluster.getNameNodeInfos()[0].setStartOpt(builder.startOpt);
+    cluster.getNameNodeInfos()[1].setStartOpt(builder.startOpt);
+    
     // restart the cluster
     // restart the cluster
     cluster.restartNameNodes();
     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;
 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.BufferedReader;
 import java.io.File;
 import java.io.File;
 import java.io.FileReader;
 import java.io.FileReader;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.InputStream;
 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 java.util.regex.Pattern;
 
 
 import org.apache.commons.logging.impl.Log4JLogger;
 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.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.PathUtils;
 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.Level;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
 import org.apache.log4j.Logger;
@@ -54,13 +57,30 @@ import org.apache.log4j.RollingFileAppender;
 import org.junit.After;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Test;
 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
  * A JUnit test that audit logs are generated
  */
  */
+@RunWith(Parameterized.class)
 public class TestAuditLogs {
 public class TestAuditLogs {
   static final String auditLogFile = PathUtils.getTestDirName(TestAuditLogs.class) + "/TestAuditLogs-audit.log";
   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: 
   // Pattern for: 
   // allowed=(true|false) ugi=name ip=/address cmd={cmd} src={path} dst=null perm=null
   // allowed=(true|false) ugi=name ip=/address cmd={cmd} src={path} dst=null perm=null
   static final Pattern auditPattern = Pattern.compile(
   static final Pattern auditPattern = Pattern.compile(
@@ -84,17 +104,28 @@ public class TestAuditLogs {
 
 
   @Before
   @Before
   public void setupCluster() throws Exception {
   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();
     conf = new HdfsConfiguration();
     final long precision = 1L;
     final long precision = 1L;
     conf.setLong(DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY, precision);
     conf.setLong(DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY, precision);
     conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 10000L);
     conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 10000L);
     conf.setBoolean(DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY, true);
     conf.setBoolean(DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY, true);
+    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_AUDIT_LOG_ASYNC_KEY, useAsyncLog);
     util = new DFSTestUtil.Builder().setName("TestAuditAllowed").
     util = new DFSTestUtil.Builder().setName("TestAuditAllowed").
         setNumFiles(20).build();
         setNumFiles(20).build();
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(4).build();
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(4).build();
     fs = cluster.getFileSystem();
     fs = cluster.getFileSystem();
     util.createFiles(fs, fileName);
     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);
     fnames = util.getFileNames(fileName);
     util.waitReplication(fs, fileName, (short)3);
     util.waitReplication(fs, fileName, (short)3);
     userGroupInfo = UserGroupInformation.createUserForTesting(username, groups);
     userGroupInfo = UserGroupInformation.createUserForTesting(username, groups);
@@ -203,6 +234,7 @@ public class TestAuditLogs {
     try {
     try {
       hftpFs = (HftpFileSystem) new Path(hftpUri).getFileSystem(conf);
       hftpFs = (HftpFileSystem) new Path(hftpUri).getFileSystem(conf);
       InputStream istream = hftpFs.open(file);
       InputStream istream = hftpFs.open(file);
+      @SuppressWarnings("unused")
       int val = istream.read();
       int val = istream.read();
       istream.close();
       istream.close();
 
 
@@ -234,6 +266,12 @@ public class TestAuditLogs {
 
 
   /** Sets up log4j logger for auditlogs */
   /** Sets up log4j logger for auditlogs */
   private void setupAuditLogs() throws IOException {
   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.
     // Shutdown the LogManager to release all logger open file handles.
     // Unfortunately, Apache commons logging library does not provide
     // Unfortunately, Apache commons logging library does not provide
     // means to release underlying loggers. For additional info look up
     // means to release underlying loggers. For additional info look up
@@ -245,7 +283,8 @@ public class TestAuditLogs {
       assertTrue(file.delete());
       assertTrue(file.delete());
     }
     }
     Logger logger = ((Log4JLogger) FSNamesystem.auditLog).getLogger();
     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");
     PatternLayout layout = new PatternLayout("%m%n");
     RollingFileAppender appender = new RollingFileAppender(layout, auditLogFile);
     RollingFileAppender appender = new RollingFileAppender(layout, auditLogFile);
     logger.addAppender(appender);
     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.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 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.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.junit.Test;
 import org.junit.Test;
 
 
@@ -191,6 +193,29 @@ public class TestGenericJournalConf {
       shouldPromptCalled = true;
       shouldPromptCalled = true;
       return false;
       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 {
   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.FileNotFoundException;
 import java.io.IOException;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Arrays;
 import java.util.List;
 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
   @Test
   public void testFilesInGetListingOps() throws Exception {
   public void testFilesInGetListingOps() throws Exception {
     final Configuration conf = new Configuration();
     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");
       fail("Did not throw");
     } catch (IOException ioe) {
     } catch (IOException ioe) {
       GenericTestUtils.assertExceptionContains(
       GenericTestUtils.assertExceptionContains(
-          "Cannot start an HA namenode with name dirs that need recovery",
+          "storage directory does not exist or is not accessible",
           ioe);
           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;
 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.assertTrue;
 import static org.junit.Assert.fail;
 import static org.junit.Assert.fail;
 
 
 import java.io.File;
 import java.io.File;
 import java.io.IOException;
 import java.io.IOException;
 import java.net.URI;
 import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Collection;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 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.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 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.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.Storage;
 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.apache.hadoop.test.GenericTestUtils;
+import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
 
 
-import com.google.common.collect.Lists;
+import com.google.common.base.Joiner;
 
 
 /**
 /**
  * Tests for upgrading with HA enabled.
  * Tests for upgrading with HA enabled.
  */
  */
 public class TestDFSUpgradeWithHA {
 public class TestDFSUpgradeWithHA {
-  
+
   private static final Log LOG = LogFactory.getLog(TestDFSUpgradeWithHA.class);
   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
   @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 {
       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(
         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
   @Test
-  public void testStartingWithUpgradeInProgressFails() throws Exception {
+  public void testNfsUpgrade() throws IOException, URISyntaxException {
     MiniDFSCluster cluster = null;
     MiniDFSCluster cluster = null;
+    FileSystem fs = null;
     try {
     try {
-      cluster = new MiniDFSCluster.Builder(new Configuration())
+      cluster = new MiniDFSCluster.Builder(conf)
           .nnTopology(MiniDFSNNTopology.simpleHATopology())
           .nnTopology(MiniDFSNNTopology.simpleHATopology())
           .numDataNodes(0)
           .numDataNodes(0)
           .build();
           .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.
       // Simulate an upgrade having started.
       for (int i = 0; i < 2; i++) {
       for (int i = 0; i < 2; i++) {
         for (URI uri : cluster.getNameDirs(i)) {
         for (URI uri : cluster.getNameDirs(i)) {
@@ -92,18 +509,226 @@ public class TestDFSUpgradeWithHA {
           assertTrue(prevTmp.mkdirs());
           assertTrue(prevTmp.mkdirs());
         }
         }
       }
       }
-      
+
       cluster.restartNameNodes();
       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 {
     } finally {
       if (cluster != null) {
       if (cluster != null) {
         cluster.shutdown();
         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) {
     } catch (IOException ioe) {
       LOG.info("Got expected exception", ioe);
       LOG.info("Got expected exception", ioe);
       GenericTestUtils.assertExceptionContains(
       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 {
     try {
       cluster.restartNameNode(1, false);
       cluster.restartNameNode(1, false);
@@ -104,7 +104,7 @@ public class TestInitializeSharedEdits {
     } catch (IOException ioe) {
     } catch (IOException ioe) {
       LOG.info("Got expected exception", ioe);
       LOG.info("Got expected exception", ioe);
       GenericTestUtils.assertExceptionContains(
       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;
 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.apache.hadoop.fs.FileStatus;
 import org.junit.After;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
 import static org.hamcrest.CoreMatchers.is;
 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.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.util.ToolRunner;
 
 
 public class TestSnapshotFileLength {
 public class TestSnapshotFileLength {
 
 
@@ -112,4 +116,61 @@ public class TestSnapshotFileLength {
     assertThat(bytesRead, is(BLOCKSIZE));
     assertThat(bytesRead, is(BLOCKSIZE));
     fis.close();
     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
     MAPREDUCE-5672. Provide optional RollingFileAppender for container log4j
     (syslog) (Gera Shegalov via jlowe)
     (syslog) (Gera Shegalov via jlowe)
 
 
+    MAPREDUCE-5725. Make explicit that TestNetworkedJob relies on the Capacity
+    Scheduler (Sandy Ryza)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
     MAPREDUCE-5484. YarnChild unnecessarily loads job conf twice (Sandy Ryza)
     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. 
     MAPREDUCE-5724. JobHistoryServer does not start if HDFS is not running. 
     (tucu)
     (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
 Release 2.3.0 - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   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.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.List;
+import java.util.concurrent.ScheduledExecutorService;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 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.ApplicationConstants.Environment;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.ConverterUtils;
-import org.apache.log4j.LogManager;
 
 
 /**
 /**
  * The main() for MapReduce task processes.
  * The main() for MapReduce task processes.
@@ -123,6 +123,7 @@ class YarnChild {
     LOG.debug("PID: " + System.getenv().get("JVM_PID"));
     LOG.debug("PID: " + System.getenv().get("JVM_PID"));
     Task task = null;
     Task task = null;
     UserGroupInformation childUGI = null;
     UserGroupInformation childUGI = null;
+    ScheduledExecutorService logSyncer = null;
 
 
     try {
     try {
       int idleLoopCount = 0;
       int idleLoopCount = 0;
@@ -161,6 +162,8 @@ class YarnChild {
       // set job classloader if configured before invoking the task
       // set job classloader if configured before invoking the task
       MRApps.setJobClassLoader(job);
       MRApps.setJobClassLoader(job);
 
 
+      logSyncer = TaskLog.createLogSyncer();
+
       // Create a final reference to the task for the doAs block
       // Create a final reference to the task for the doAs block
       final Task taskFinal = task;
       final Task taskFinal = task;
       childUGI.doAs(new PrivilegedExceptionAction<Object>() {
       childUGI.doAs(new PrivilegedExceptionAction<Object>() {
@@ -214,10 +217,7 @@ class YarnChild {
     } finally {
     } finally {
       RPC.stopProxy(umbilical);
       RPC.stopProxy(umbilical);
       DefaultMetricsSystem.shutdown();
       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.Map.Entry;
 import java.util.Set;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 
 import org.apache.commons.io.IOUtils;
 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.JobConf;
 import org.apache.hadoop.mapred.LocalContainerLauncher;
 import org.apache.hadoop.mapred.LocalContainerLauncher;
 import org.apache.hadoop.mapred.TaskAttemptListenerImpl;
 import org.apache.hadoop.mapred.TaskAttemptListenerImpl;
+import org.apache.hadoop.mapred.TaskLog;
 import org.apache.hadoop.mapred.TaskUmbilicalProtocol;
 import org.apache.hadoop.mapred.TaskUmbilicalProtocol;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.OutputCommitter;
 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.CompositeService;
 import org.apache.hadoop.service.Service;
 import org.apache.hadoop.service.Service;
 import org.apache.hadoop.service.ServiceOperations;
 import org.apache.hadoop.service.ServiceOperations;
+import org.apache.hadoop.util.ExitUtil;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.ShutdownHookManager;
 import org.apache.hadoop.util.ShutdownHookManager;
 import org.apache.hadoop.util.StringInterner;
 import org.apache.hadoop.util.StringInterner;
@@ -212,6 +215,7 @@ public class MRAppMaster extends CompositeService {
   boolean errorHappenedShutDown = false;
   boolean errorHappenedShutDown = false;
   private String shutDownMessage = null;
   private String shutDownMessage = null;
   JobStateInternal forcedState = null;
   JobStateInternal forcedState = null;
+  private final ScheduledExecutorService logSyncer;
 
 
   private long recoveredJobStartTime = 0;
   private long recoveredJobStartTime = 0;
 
 
@@ -240,6 +244,7 @@ public class MRAppMaster extends CompositeService {
     this.nmHttpPort = nmHttpPort;
     this.nmHttpPort = nmHttpPort;
     this.metrics = MRAppMetrics.create();
     this.metrics = MRAppMetrics.create();
     this.maxAppAttempts = maxAppAttempts;
     this.maxAppAttempts = maxAppAttempts;
+    logSyncer = TaskLog.createLogSyncer();
     LOG.info("Created MRAppMaster for application " + applicationAttemptId);
     LOG.info("Created MRAppMaster for application " + applicationAttemptId);
   }
   }
 
 
@@ -1078,6 +1083,12 @@ public class MRAppMaster extends CompositeService {
     // All components have started, start the job.
     // All components have started, start the job.
     startJobs();
     startJobs();
   }
   }
+  
+  @Override
+  public void stop() {
+    super.stop();
+    TaskLog.syncLogsShutdown(logSyncer);
+  }
 
 
   private void processRecovery() {
   private void processRecovery() {
     if (appAttemptID.getAttemptId() == 1) {
     if (appAttemptID.getAttemptId() == 1) {
@@ -1395,9 +1406,7 @@ public class MRAppMaster extends CompositeService {
       initAndStartAppMaster(appMaster, conf, jobUserName);
       initAndStartAppMaster(appMaster, conf, jobUserName);
     } catch (Throwable t) {
     } catch (Throwable t) {
       LOG.fatal("Error starting MRAppMaster", 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.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 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.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueACL;
@@ -445,11 +446,18 @@ public class TypeConverter {
     jobStatus.setStartTime(application.getStartTime());
     jobStatus.setStartTime(application.getStartTime());
     jobStatus.setFinishTime(application.getFinishTime());
     jobStatus.setFinishTime(application.getFinishTime());
     jobStatus.setFailureInfo(application.getDiagnostics());
     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;
     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.ArrayList;
 import java.util.List;
 import java.util.List;
 
 
-import junit.framework.Assert;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.JobStatus.State;
 import org.apache.hadoop.mapreduce.JobStatus.State;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 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.Resource;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.util.Records;
 import org.apache.hadoop.yarn.util.Records;
+import org.junit.Assert;
 import org.junit.Test;
 import org.junit.Test;
 import org.mockito.Mockito;
 import org.mockito.Mockito;
 
 
@@ -112,6 +111,14 @@ public class TestTypeConverter {
     when(mockReport.getUser()).thenReturn("dummy-user");
     when(mockReport.getUser()).thenReturn("dummy-user");
     when(mockReport.getQueue()).thenReturn("dummy-queue");
     when(mockReport.getQueue()).thenReturn("dummy-queue");
     String jobFile = "dummy-path/job.xml";
     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
     ApplicationResourceUsageReport appUsageRpt = Records
         .newRecord(ApplicationResourceUsageReport.class);
         .newRecord(ApplicationResourceUsageReport.class);
     Resource r = Records.newRecord(Resource.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.DataOutputStream;
 import java.io.File;
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileInputStream;
+import java.io.Flushable;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.io.InputStreamReader;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Enumeration;
 import java.util.Enumeration;
 import java.util.List;
 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.Log;
 import org.apache.commons.logging.LogFactory;
 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.JobID;
 import org.apache.hadoop.mapreduce.util.ProcessTree;
 import org.apache.hadoop.mapreduce.util.ProcessTree;
 import org.apache.hadoop.util.Shell;
 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.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.log4j.Appender;
 import org.apache.log4j.Appender;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.LogManager;
@@ -262,7 +269,86 @@ public class TaskLog {
     }
     }
     writeToIndexFile(logLocation, isCleanup);
     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.
    * 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;
 package org.apache.hadoop.mapred;
 
 
+import java.io.Flushable;
 import java.util.LinkedList;
 import java.util.LinkedList;
 import java.util.Queue;
 import java.util.Queue;
 
 
@@ -31,7 +32,7 @@ import org.apache.log4j.spi.LoggingEvent;
  * 
  * 
  */
  */
 @InterfaceStability.Unstable
 @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
   private String taskId; //taskId should be managed as String rather than TaskID object
   //so that log4j can configure it from the configuration(log4j.properties). 
   //so that log4j can configure it from the configuration(log4j.properties). 
   private Integer maxEvents;
   private Integer maxEvents;
@@ -92,6 +93,7 @@ public class TaskLogAppender extends FileAppender {
     }
     }
   }
   }
   
   
+  @Override
   public void flush() {
   public void flush() {
     if (qw != null) {
     if (qw != null) {
       qw.flush();
       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.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
 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.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 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.NodeReport;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.NodeState;
@@ -371,4 +375,29 @@ public class ResourceMgrDelegate extends YarnClient {
       IOException {
       IOException {
     return client.getQueueAclsInfo();
     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.mapred.lib.IdentityReducer;
 import org.apache.hadoop.mapreduce.Cluster.JobTrackerStatus;
 import org.apache.hadoop.mapreduce.Cluster.JobTrackerStatus;
 import org.apache.hadoop.mapreduce.Job;
 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.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
 import org.junit.Test;
 import org.junit.Test;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
@@ -76,8 +78,7 @@ public class TestNetworkedJob {
     FileSystem fileSys = null;
     FileSystem fileSys = null;
 
 
     try {
     try {
-      mr = MiniMRClientClusterFactory.create(this.getClass(), 2,
-          new Configuration());
+      mr = createMiniClusterWithCapacityScheduler();
 
 
       JobConf job = new JobConf(mr.getConfig());
       JobConf job = new JobConf(mr.getConfig());
 
 
@@ -129,8 +130,7 @@ public class TestNetworkedJob {
     FileSystem fileSys = null;
     FileSystem fileSys = null;
 
 
     try {
     try {
-      Configuration conf = new Configuration();
-      mr = MiniMRClientClusterFactory.create(this.getClass(), 2, conf);
+      mr = createMiniClusterWithCapacityScheduler();
 
 
       JobConf job = new JobConf(mr.getConfig());
       JobConf job = new JobConf(mr.getConfig());
 
 
@@ -315,8 +315,7 @@ public class TestNetworkedJob {
     FileSystem fileSys = null;
     FileSystem fileSys = null;
     PrintStream oldOut = System.out;
     PrintStream oldOut = System.out;
     try {
     try {
-      Configuration conf = new Configuration();
-      mr = MiniMRClientClusterFactory.create(this.getClass(), 2, conf);
+      mr = createMiniClusterWithCapacityScheduler();
 
 
       JobConf job = new JobConf(mr.getConfig());
       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>
   <url>http://hadoop.apache.org/mapreduce/</url>
 
 
   <properties>
   <properties>
-    <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
     <test.logs>true</test.logs>
     <test.logs>true</test.logs>
     <test.timeout>600000</test.timeout>
     <test.timeout>600000</test.timeout>
     <fork.mode>once</fork.mode>
     <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 -->
     <!-- number of threads/forks to use when running tests in parallel, see parallel-tests profile -->
     <testsThreadCount>4</testsThreadCount>
     <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 -->
     <!-- These 2 versions are defined here becuase they are used -->
     <!-- JDIFF generation from embedded ant in the antrun plugin -->
     <!-- JDIFF generation from embedded ant in the antrun plugin -->
     <jdiff.version>1.0.9</jdiff.version>
     <jdiff.version>1.0.9</jdiff.version>

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

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

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