Browse Source

Merge trunk into auto-failover branch.

Addressed two semantic conflicts after the commit of HADOOP-8077 -- test code was referencing the old constant for the fencing methods config key.


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-3042@1310174 13f79535-47bb-0310-9956-ffa450edef68
Todd Lipcon 13 years ago
parent
commit
30068a1d83
90 changed files with 2708 additions and 2810 deletions
  1. 2 2
      dev-support/test-patch.sh
  2. 19 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  3. 5 5
      hadoop-common-project/hadoop-common/pom.xml
  4. 16 6
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ByteBufferReadable.java
  5. 9 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
  6. 1 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CopyCommands.java
  7. 8 11
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/NodeFencer.java
  8. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java
  9. 17 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileUtil.java
  10. 2 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/DummyHAService.java
  11. 1 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestNodeFencer.java
  12. 2 4
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestShellCommandFencer.java
  13. 1 1
      hadoop-hdfs-project/dev-support/test-patch.properties
  14. 0 8
      hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml
  15. 24 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  16. 10 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  17. 14 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java
  18. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java
  19. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolServerSideTranslatorPB.java
  20. 2 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolTranslatorPB.java
  21. 25 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
  22. 8 13
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
  23. 47 12
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
  24. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
  25. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  26. 64 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/journalservice/JournalListener.java
  27. 97 128
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/journalservice/JournalService.java
  28. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupImage.java
  29. 27 12
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
  30. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java
  31. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java
  32. 580 24
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
  33. 23 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  34. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileChecksumServlets.java
  35. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java
  36. 90 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
  37. 5 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java
  38. 11 15
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeRegistration.java
  39. 9 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java
  40. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/NNHAServiceTarget.java
  41. 17 70
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/BinaryEditsVisitor.java
  42. 0 88
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsElement.java
  43. 0 74
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsLoader.java
  44. 0 608
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsLoaderCurrent.java
  45. 0 175
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsVisitor.java
  46. 0 55
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsVisitorFactory.java
  47. 83 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsBinaryLoader.java
  48. 62 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsLoader.java
  49. 54 50
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsViewer.java
  50. 21 29
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsVisitor.java
  51. 77 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsVisitorFactory.java
  52. 256 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsXmlLoader.java
  53. 26 106
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/StatisticsEditsVisitor.java
  54. 40 19
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TeeOutputStream.java
  55. 0 109
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TextEditsVisitor.java
  56. 0 401
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/Tokenizer.java
  57. 65 113
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/XmlEditsVisitor.java
  58. 0 140
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/XmlTokenizer.java
  59. 171 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/XMLUtils.java
  60. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
  61. 18 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/JournalProtocol.proto
  62. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
  63. 14 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
  64. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReplaceDatanodeOnFailure.java
  65. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java
  66. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/journalservice/TestJournalService.java
  67. 5 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
  68. 22 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestValidateConfigurationSettings.java
  69. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestBootstrapStandby.java
  70. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDFSZKFailoverController.java
  71. 120 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestInitializeSharedEdits.java
  72. 41 7
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdmin.java
  73. 6 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdminMiniCluster.java
  74. 9 14
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TestOfflineEditsViewer.java
  75. 14 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsFileSystemContract.java
  76. 315 364
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml
  77. 10 0
      hadoop-mapreduce-project/CHANGES.txt
  78. 0 15
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/pom.xml
  79. 6 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/pom.xml
  80. 5 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDataDrivenDBInputFormat.java
  81. 0 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml
  82. 5 0
      hadoop-mapreduce-project/hadoop-mapreduce-examples/pom.xml
  83. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/DBCountPageView.java
  84. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-examples/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDBJob.java
  85. 5 1
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSAssignment.java
  86. 5 8
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java
  87. 65 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java
  88. 0 4
      hadoop-mapreduce-project/hadoop-yarn/pom.xml
  89. 0 4
      hadoop-mapreduce-project/pom.xml
  90. 20 5
      hadoop-project/pom.xml

+ 2 - 2
dev-support/test-patch.sh

@@ -311,7 +311,7 @@ checkTests () {
   echo "======================================================================"
   echo ""
   echo ""
-  testReferences=`$GREP -c -i '/test' $PATCH_DIR/patch`
+  testReferences=`$GREP -c -i -e '^+++.*/test' $PATCH_DIR/patch`
   echo "There appear to be $testReferences test files referenced in the patch."
   if [[ $testReferences == 0 ]] ; then
     if [[ $JENKINS == "true" ]] ; then
@@ -333,7 +333,7 @@ checkTests () {
   fi
   JIRA_COMMENT="$JIRA_COMMENT
 
-    +1 tests included.  The patch appears to include $testReferences new or modified tests."
+    +1 tests included.  The patch appears to include $testReferences new or modified test files."
   return 0
 }
 

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

@@ -52,6 +52,11 @@ Trunk (unreleased changes)
     HADOOP-7994. Remove getProtocolVersion and getProtocolSignature from the 
     client side translator and server side implementation. (jitendra)
 
+    HADOOP-8244. Improve comments on ByteBufferReadable.read. (Henry Robinson
+    via atm)
+
+    HADOOP-7757. Test file reference count is at least 3x actual value (Jon
+    Eagles via bobby)
 
   BUG FIXES
 
@@ -247,6 +252,9 @@ Release 2.0.0 - UNRELEASED
 
     HADOOP-8007. Use substitution tokens for fencing argument (todd)
 
+    HADOOP-8077. HA: fencing method should be able to be configured on
+    a per-NN or per-NS basis (todd)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -319,6 +327,8 @@ Release 2.0.0 - UNRELEASED
     HADOOP-8243. Security support broken in CLI (manual) failover controller
     (todd)
 
+    HADOOP-8251. Fix SecurityUtil.fetchServiceTicket after HADOOP-6941 (todd)
+
   BREAKDOWN OF HADOOP-7454 SUBTASKS
 
     HADOOP-7455. HA: Introduce HA Service Protocol Interface. (suresh)
@@ -386,6 +396,15 @@ Release 0.23.3 - UNRELEASED
     HADOOP-8088. User-group mapping cache incorrectly does negative caching on
     transient failures (Khiwal Lee via bobby)
 
+    HADOOP-8179. risk of NPE in CopyCommands processArguments() (Daryn Sharp
+    via bobby)
+
+    HADOOP-6963. In FileUtil.getDU(..), neither include the size of directories
+    nor follow symbolic links.  (Ravi Prakash via szetszwo)
+
+    HADOOP-8180. Remove hsqldb since its not needed from pom.xml (Ravi Prakash
+    via tgraves)
+
 Release 0.23.2 - UNRELEASED 
 
   INCOMPATIBLE CHANGES

+ 5 - 5
hadoop-common-project/hadoop-common/pom.xml

@@ -78,6 +78,11 @@
       <artifactId>commons-net</artifactId>
       <scope>compile</scope>
     </dependency>
+    <dependency>
+      <groupId>commons-io</groupId>
+      <artifactId>commons-io</artifactId>
+      <scope>compile</scope>
+    </dependency>
     <dependency>
       <groupId>javax.servlet</groupId>
       <artifactId>servlet-api</artifactId>
@@ -193,11 +198,6 @@
       <artifactId>commons-configuration</artifactId>
       <scope>compile</scope>
     </dependency>
-    <dependency>
-      <groupId>hsqldb</groupId>
-      <artifactId>hsqldb</artifactId>
-      <scope>compile</scope>
-    </dependency>
     <dependency>
       <groupId>org.slf4j</groupId>
       <artifactId>slf4j-api</artifactId>

+ 16 - 6
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ByteBufferReadable.java

@@ -28,18 +28,28 @@ public interface ByteBufferReadable {
   /**
    * Reads up to buf.remaining() bytes into buf. Callers should use
    * buf.limit(..) to control the size of the desired read.
-   *
-   * After the call, buf.position() should be unchanged, and therefore any data
-   * can be immediately read from buf.
-   *
+   * <p/>
+   * After a successful call, buf.position() and buf.limit() should be
+   * unchanged, and therefore any data can be immediately read from buf.
+   * buf.mark() may be cleared or updated.
+   * <p/>
+   * In the case of an exception, the values of buf.position() and buf.limit()
+   * are undefined, and callers should be prepared to recover from this
+   * eventuality.
+   * <p/>
    * Many implementations will throw {@link UnsupportedOperationException}, so
    * callers that are not confident in support for this method from the
    * underlying filesystem should be prepared to handle that exception.
+   * <p/>
+   * Implementations should treat 0-length requests as legitimate, and must not
+   * signal an error upon their receipt.
    *
    * @param buf
-   *          the ByteBuffer to receive the results of the read operation
+   *          the ByteBuffer to receive the results of the read operation. Up to
+   *          buf.limit() - buf.position() bytes may be read.
    * @return the number of bytes available to read from buf
-   * @throws IOException if there is some error performing the read
+   * @throws IOException
+   *           if there is some error performing the read
    */
   public int read(ByteBuffer buf) throws IOException;
 }

+ 9 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java

@@ -483,11 +483,18 @@ public class FileUtil {
     if (!dir.isDirectory()) {
       return dir.length();
     } else {
-      size = dir.length();
       File[] allFiles = dir.listFiles();
       if(allFiles != null) {
          for (int i = 0; i < allFiles.length; i++) {
-            size = size + getDU(allFiles[i]);
+           boolean isSymLink;
+           try {
+             isSymLink = org.apache.commons.io.FileUtils.isSymlink(allFiles[i]);
+           } catch(IOException ioe) {
+             isSymLink = true;
+           }
+           if(!isSymLink) {
+             size += getDU(allFiles[i]);
+           }
          }
       }
       return size;

+ 1 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CopyCommands.java

@@ -81,10 +81,9 @@ class CopyCommands {
       }
       FSDataOutputStream out = dst.fs.create(dst.path);
       try {
-        FSDataInputStream in = null;
         for (PathData src : srcs) {
+          FSDataInputStream in = src.fs.open(src.path);
           try {
-            in = src.fs.open(src.path);
             IOUtils.copyBytes(in, out, getConf(), false);
             if (delimiter != null) {
               out.write(delimiter.getBytes("UTF-8"));

+ 8 - 11
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/NodeFencer.java

@@ -53,9 +53,6 @@ import com.google.common.collect.Lists;
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 public class NodeFencer {
-  public static final String CONF_METHODS_KEY =
-    "dfs.ha.fencing.methods";
-  
   private static final String CLASS_RE = "([a-zA-Z0-9\\.\\$]+)";
   private static final Pattern CLASS_WITH_ARGUMENT =
     Pattern.compile(CLASS_RE + "\\((.+?)\\)");
@@ -76,18 +73,18 @@ public class NodeFencer {
   
   private final List<FenceMethodWithArg> methods;
   
-  public NodeFencer(Configuration conf)
+  NodeFencer(Configuration conf, String spec)
       throws BadFencingConfigurationException {
-    this.methods = parseMethods(conf);
+    this.methods = parseMethods(conf, spec);
   }
   
-  public static NodeFencer create(Configuration conf)
+  public static NodeFencer create(Configuration conf, String confKey)
       throws BadFencingConfigurationException {
-    String confStr = conf.get(CONF_METHODS_KEY);
+    String confStr = conf.get(confKey);
     if (confStr == null) {
       return null;
     }
-    return new NodeFencer(conf);
+    return new NodeFencer(conf, confStr);
   }
 
   public boolean fence(HAServiceTarget fromSvc) {
@@ -115,10 +112,10 @@ public class NodeFencer {
     return false;
   }
 
-  private static List<FenceMethodWithArg> parseMethods(Configuration conf)
+  private static List<FenceMethodWithArg> parseMethods(Configuration conf,
+      String spec)
       throws BadFencingConfigurationException {
-    String confStr = conf.get(CONF_METHODS_KEY);
-    String[] lines = confStr.split("\\s*\n\\s*");
+    String[] lines = spec.split("\\s*\n\\s*");
     
     List<FenceMethodWithArg> methods = Lists.newArrayList();
     for (String line : lines) {

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

@@ -171,7 +171,7 @@ public class SecurityUtil {
       } else {
         principalClass = Class.forName("sun.security.krb5.PrincipalName");
         credentialsClass = Class.forName("sun.security.krb5.Credentials");
-        krb5utilClass = Class.forName("sun.security.jgss.krb5");
+        krb5utilClass = Class.forName("sun.security.jgss.krb5.Krb5Util");
       }
       @SuppressWarnings("rawtypes")
       Constructor principalConstructor = principalClass.getConstructor(String.class, 

+ 17 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileUtil.java

@@ -93,6 +93,9 @@ public class TestFileUtil {
     // create files in partitioned directories
     createFile(partitioned, "part-r-00000", "foo");
     createFile(partitioned, "part-r-00001", "bar");
+
+    // create a cycle using symlinks. Cycles should be handled
+    FileUtil.symLink(del.toString(), dir1.toString() + "/cycle");
   }
 
   /**
@@ -458,4 +461,18 @@ public class TestFileUtil {
 
     return result;
   }
+
+  /**
+   * Test that getDU is able to handle cycles caused due to symbolic links
+   * and that directory sizes are not added to the final calculated size
+   * @throws IOException
+   */
+  @Test
+  public void testGetDU() throws IOException {
+    setupDirs();
+
+    long du = FileUtil.getDU(TEST_DIR);
+    //Only two files (in partitioned) have 4 bytes each
+    Assert.assertEquals(du, 8);
+  }
 }

+ 2 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/DummyHAService.java

@@ -55,8 +55,8 @@ class DummyHAService extends HAServiceTarget {
     this.proxy = makeMock();
     try {
       Configuration conf = new Configuration();
-      conf.set(NodeFencer.CONF_METHODS_KEY, DummyFencer.class.getName());
-      this.fencer = Mockito.spy(NodeFencer.create(conf));
+      this.fencer = Mockito.spy(NodeFencer.create(conf,
+          DummyFencer.class.getName()));
     } catch (BadFencingConfigurationException e) {
       throw new RuntimeException(e);
     }

+ 1 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestNodeFencer.java

@@ -132,8 +132,7 @@ public class TestNodeFencer {
       throws BadFencingConfigurationException {
     System.err.println("Testing configuration:\n" + confStr);
     Configuration conf = new Configuration();
-    conf.set(NodeFencer.CONF_METHODS_KEY, confStr);
-    return new NodeFencer(conf);
+    return new NodeFencer(conf, confStr);
   }
   
   /**

+ 2 - 4
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestShellCommandFencer.java

@@ -71,8 +71,7 @@ public class TestShellCommandFencer {
   public void testCheckNoArgs() {
     try {
       Configuration conf = new Configuration();
-      conf.set(NodeFencer.CONF_METHODS_KEY, "shell");
-      new NodeFencer(conf);
+      new NodeFencer(conf, "shell");
       fail("Didn't throw when passing no args to shell");
     } catch (BadFencingConfigurationException confe) {
       assertTrue(
@@ -85,8 +84,7 @@ public class TestShellCommandFencer {
   public void testCheckParensNoArgs() {
     try {
       Configuration conf = new Configuration();
-      conf.set(NodeFencer.CONF_METHODS_KEY, "shell()");
-      new NodeFencer(conf);
+      new NodeFencer(conf, "shell()");
       fail("Didn't throw when passing no args to shell");
     } catch (BadFencingConfigurationException confe) {
       assertTrue(

+ 1 - 1
hadoop-hdfs-project/dev-support/test-patch.properties

@@ -18,4 +18,4 @@
 
 OK_RELEASEAUDIT_WARNINGS=0
 OK_FINDBUGS_WARNINGS=0
-OK_JAVADOC_WARNINGS=0
+OK_JAVADOC_WARNINGS=8

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

@@ -145,10 +145,6 @@
           <groupId>net.java.dev.jets3t</groupId>
           <artifactId>jets3t</artifactId>
         </exclusion>
-        <exclusion>
-          <groupId>hsqldb</groupId>
-          <artifactId>hsqldb</artifactId>
-        </exclusion>
         <exclusion>
           <groupId>org.eclipse.jdt</groupId>
           <artifactId>core</artifactId>
@@ -212,10 +208,6 @@
           <groupId>net.java.dev.jets3t</groupId>
           <artifactId>jets3t</artifactId>
         </exclusion>
-        <exclusion>
-          <groupId>hsqldb</groupId>
-          <artifactId>hsqldb</artifactId>
-        </exclusion>
         <exclusion>
           <groupId>org.eclipse.jdt</groupId>
           <artifactId>core</artifactId>

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

@@ -59,6 +59,12 @@ Trunk (unreleased changes)
 
     HDFS-3131. Improve TestStorageRestore. (Brandon Li via atm)
 
+    HDFS-3178. Add states and state handler for journal synchronization in
+    JournalService.  (szetszwo)
+
+    HDFS-3204. Minor modification to JournalProtocol.proto to make
+    it generic. (suresh)
+
   OPTIMIZATIONS
 
     HDFS-2834. Add a ByteBuffer-based read API to DFSInputStream.
@@ -184,6 +190,8 @@ Release 2.0.0 - UNRELEASED
 
     HDFS-3000. Add a public API for setting quotas. (atm)
 
+    HDFS-3102. Add CLI tool to initialize the shared-edits dir. (atm)
+
   IMPROVEMENTS
 
     HDFS-2018. Move all journal stream management code into one place.
@@ -316,6 +324,9 @@ Release 2.0.0 - UNRELEASED
     HDFS-3084. FenceMethod.tryFence() and ShellCommandFencer should pass
     namenodeId as well as host:port (todd)
 
+    HDFS-3050. rework OEV to share more code with the NameNode.
+    (Colin Patrick McCabe via eli)
+
   OPTIMIZATIONS
 
     HDFS-3024. Improve performance of stringification in addStoredBlock (todd)
@@ -412,6 +423,19 @@ Release 2.0.0 - UNRELEASED
 
     HDFS-3174. Fix assert in TestPendingDataNodeMessages. (eli)
 
+    HDFS-3199. TestValidateConfigurationSettings is failing. (todd via eli)
+
+    HDFS-3202. NamespaceInfo PB translation drops build version. (atm)
+
+    HDFS-3109. Remove hsqldf exclusions from pom.xml. (Ravi Prakash
+    via suresh)
+
+    HDFS-3210. JsonUtil#toJsonMap for for a DatanodeInfo should use
+    "ipAddr" instead of "name". (eli)
+
+    HDFS-3208. Bogus entries in hosts files are incorrectly displayed
+    in the report. (eli)
+
   BREAKDOWN OF HDFS-1623 SUBTASKS
 
     HDFS-2179. Add fencing framework and mechanisms for NameNode HA. (todd)

+ 10 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java

@@ -211,7 +211,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_BALANCER_MOVEDWINWIDTH_KEY = "dfs.balancer.movedWinWidth";
   public static final long    DFS_BALANCER_MOVEDWINWIDTH_DEFAULT = 5400*1000L;
   public static final String  DFS_DATANODE_ADDRESS_KEY = "dfs.datanode.address";
-  public static final String  DFS_DATANODE_ADDRESS_DEFAULT = "0.0.0.0:50010";
+  public static final int     DFS_DATANODE_DEFAULT_PORT = 50010;
+  public static final String  DFS_DATANODE_ADDRESS_DEFAULT = "0.0.0.0:" + DFS_DATANODE_DEFAULT_PORT;
   public static final String  DFS_DATANODE_DATA_DIR_PERMISSION_KEY = "dfs.datanode.data.dir.perm";
   public static final String  DFS_DATANODE_DATA_DIR_PERMISSION_DEFAULT = "700";
   public static final String  DFS_DATANODE_DIRECTORYSCAN_INTERVAL_KEY = "dfs.datanode.directoryscan.interval";
@@ -227,7 +228,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_DATANODE_HANDLER_COUNT_KEY = "dfs.datanode.handler.count";
   public static final int     DFS_DATANODE_HANDLER_COUNT_DEFAULT = 3;
   public static final String  DFS_DATANODE_HTTP_ADDRESS_KEY = "dfs.datanode.http.address";
-  public static final String  DFS_DATANODE_HTTP_ADDRESS_DEFAULT = "0.0.0.0:50075";
+  public static final int     DFS_DATANODE_HTTP_DEFAULT_PORT = 50075;
+  public static final String  DFS_DATANODE_HTTP_ADDRESS_DEFAULT = "0.0.0.0:" + DFS_DATANODE_HTTP_DEFAULT_PORT;
   public static final String  DFS_DATANODE_MAX_RECEIVER_THREADS_KEY = "dfs.datanode.max.transfer.threads";
   public static final int     DFS_DATANODE_MAX_RECEIVER_THREADS_DEFAULT = 4096;
   public static final String  DFS_DATANODE_NUMBLOCKS_KEY = "dfs.datanode.numblocks";
@@ -251,13 +253,15 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_HTTPS_ENABLE_KEY = "dfs.https.enable";
   public static final boolean DFS_HTTPS_ENABLE_DEFAULT = false;
   public static final String  DFS_HTTPS_PORT_KEY = "dfs.https.port";
-  public static final int     DFS_HTTPS_PORT_DEFAULT = 50470;
   public static final String  DFS_DEFAULT_CHUNK_VIEW_SIZE_KEY = "dfs.default.chunk.view.size";
   public static final int     DFS_DEFAULT_CHUNK_VIEW_SIZE_DEFAULT = 32*1024;
   public static final String  DFS_DATANODE_HTTPS_ADDRESS_KEY = "dfs.datanode.https.address";
-  public static final String  DFS_DATANODE_HTTPS_ADDRESS_DEFAULT = "0.0.0.0:50475";
+  public static final String  DFS_DATANODE_HTTPS_PORT_KEY = "datanode.https.port";
+  public static final int     DFS_DATANODE_HTTPS_DEFAULT_PORT = 50475;
+  public static final String  DFS_DATANODE_HTTPS_ADDRESS_DEFAULT = "0.0.0.0:" + DFS_DATANODE_HTTPS_DEFAULT_PORT;
   public static final String  DFS_DATANODE_IPC_ADDRESS_KEY = "dfs.datanode.ipc.address";
-  public static final String  DFS_DATANODE_IPC_ADDRESS_DEFAULT = "0.0.0.0:50020";
+  public static final int     DFS_DATANODE_IPC_DEFAULT_PORT = 50020;
+  public static final String  DFS_DATANODE_IPC_ADDRESS_DEFAULT = "0.0.0.0" + DFS_DATANODE_IPC_DEFAULT_PORT;
 
   public static final String  DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY = "dfs.block.access.token.enable";
   public static final boolean DFS_BLOCK_ACCESS_TOKEN_ENABLE_DEFAULT = false;
@@ -341,4 +345,5 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final int DFS_HA_LOGROLL_PERIOD_DEFAULT = 2 * 60; // 2m
   public static final String DFS_HA_TAILEDITS_PERIOD_KEY = "dfs.ha.tail-edits.period";
   public static final int DFS_HA_TAILEDITS_PERIOD_DEFAULT = 60; // 1m
+  public static final String DFS_HA_FENCE_METHODS_KEY = "dfs.ha.fencing.methods";
 }

+ 14 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java

@@ -24,6 +24,7 @@ import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.WritableComparable;
 
@@ -50,14 +51,21 @@ public class DatanodeID implements WritableComparable<DatanodeID> {
   protected int infoPort;      // info server port
   protected int ipcPort;       // IPC server port
 
-  /** Equivalent to DatanodeID(""). */
-  public DatanodeID() {this("");}
+  public DatanodeID() {
+    this("", DFSConfigKeys.DFS_DATANODE_DEFAULT_PORT);
+  }
 
-  /** Equivalent to DatanodeID(ipAddr, "", -1, -1, -1). */
-  public DatanodeID(String ipAddr) {this(ipAddr, "", "", -1, -1, -1);}
+  public DatanodeID(String ipAddr, int xferPort) {
+    this(ipAddr, "", "", xferPort,
+        DFSConfigKeys.DFS_DATANODE_HTTP_DEFAULT_PORT,
+        DFSConfigKeys.DFS_DATANODE_IPC_DEFAULT_PORT);
+  }
 
-  /** Equivalent to DatanodeID(ipAddr, "", xferPort, -1, -1). */
-  public DatanodeID(String ipAddr, int xferPort) {this(ipAddr, "", "", xferPort, -1, -1);}
+  public DatanodeID(String ipAddr, String hostName, int xferPort) {
+    this(ipAddr, hostName, "", xferPort,
+        DFSConfigKeys.DFS_DATANODE_HTTP_DEFAULT_PORT,
+        DFSConfigKeys.DFS_DATANODE_IPC_DEFAULT_PORT);
+  }
 
   /**
    * DatanodeID copy constructor

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

@@ -122,12 +122,12 @@ public class DatanodeInfo extends DatanodeID implements Node {
   }
 
   /** Constructor */
-  public DatanodeInfo(final String name, final String hostName,
+  public DatanodeInfo(final String ipAddr, final String hostName,
       final String storageID, final int xferPort, final int infoPort, final int ipcPort,
       final long capacity, final long dfsUsed, final long remaining,
       final long blockPoolUsed, final long lastUpdate, final int xceiverCount,
       final String networkLocation, final AdminStates adminState) {
-    super(name, hostName, storageID, xferPort, infoPort, ipcPort);
+    super(ipAddr, hostName, storageID, xferPort, infoPort, ipcPort);
     this.capacity = capacity;
     this.dfsUsed = dfsUsed;
     this.remaining = remaining;
@@ -265,7 +265,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
     long c = getCapacity();
     long r = getRemaining();
     long u = getDfsUsed();
-    buffer.append(ipAddr);
+    buffer.append(getName());
     if (!NetworkTopology.DEFAULT_RACK.equals(location)) {
       buffer.append(" "+location);
     }

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

@@ -48,7 +48,7 @@ public class JournalProtocolServerSideTranslatorPB implements JournalProtocolPB
   public JournalResponseProto journal(RpcController unused,
       JournalRequestProto req) throws ServiceException {
     try {
-      impl.journal(PBHelper.convert(req.getRegistration()),
+      impl.journal(PBHelper.convert(req.getJournalInfo()),
           req.getFirstTxnId(), req.getNumTxns(), req.getRecords()
               .toByteArray());
     } catch (IOException e) {
@@ -62,7 +62,7 @@ public class JournalProtocolServerSideTranslatorPB implements JournalProtocolPB
   public StartLogSegmentResponseProto startLogSegment(RpcController controller,
       StartLogSegmentRequestProto req) throws ServiceException {
     try {
-      impl.startLogSegment(PBHelper.convert(req.getRegistration()),
+      impl.startLogSegment(PBHelper.convert(req.getJournalInfo()),
           req.getTxid());
     } catch (IOException e) {
       throw new ServiceException(e);

+ 2 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolTranslatorPB.java

@@ -24,12 +24,10 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentRequestProto;
-import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
 import org.apache.hadoop.hdfs.server.protocol.JournalProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
 import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtocolMetaInterface;
-import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RpcClientUtil;
 import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
@@ -63,7 +61,7 @@ public class JournalProtocolTranslatorPB implements ProtocolMetaInterface,
   public void journal(NamenodeRegistration reg, long firstTxnId,
       int numTxns, byte[] records) throws IOException {
     JournalRequestProto req = JournalRequestProto.newBuilder()
-        .setRegistration(PBHelper.convert(reg))
+        .setJournalInfo(PBHelper.convertToJournalInfo(reg))
         .setFirstTxnId(firstTxnId)
         .setNumTxns(numTxns)
         .setRecords(PBHelper.getByteString(records))
@@ -79,7 +77,7 @@ public class JournalProtocolTranslatorPB implements ProtocolMetaInterface,
   public void startLogSegment(NamenodeRegistration registration, long txid)
       throws IOException {
     StartLogSegmentRequestProto req = StartLogSegmentRequestProto.newBuilder()
-        .setRegistration(PBHelper.convert(registration))
+        .setJournalInfo(PBHelper.convertToJournalInfo(registration))
         .setTxid(txid)
         .build();
     try {

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

@@ -95,6 +95,7 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProt
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReplicaStateProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.UpgradeStatusReportProto;
+import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalInfoProto;
 import org.apache.hadoop.hdfs.security.token.block.BlockKey;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
@@ -127,7 +128,6 @@ import org.apache.hadoop.hdfs.server.protocol.RegisterCommand;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
-import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.Text;
@@ -384,7 +384,8 @@ public class PBHelper {
   public static NamespaceInfo convert(NamespaceInfoProto info) {
     StorageInfoProto storage = info.getStorageInfo();
     return new NamespaceInfo(storage.getNamespceID(), storage.getClusterID(),
-        info.getBlockPoolID(), storage.getCTime(), info.getDistUpgradeVersion());
+        info.getBlockPoolID(), storage.getCTime(), info.getDistUpgradeVersion(),
+        info.getBuildVersion());
   }
 
   public static NamenodeCommand convert(NamenodeCommandProto cmd) {
@@ -1345,4 +1346,26 @@ public class PBHelper {
         .setDfsUsed(r.getDfsUsed()).setRemaining(r.getRemaining())
         .setStorageID(r.getStorageID()).build();
   }
+
+  public static NamenodeRegistration convert(JournalInfoProto info) {
+    int lv = info.hasLayoutVersion() ? info.getLayoutVersion() : 0;
+    int nsID = info.hasNamespaceID() ? info.getNamespaceID() : 0;
+    StorageInfo storage = new StorageInfo(lv, nsID, info.getClusterID(), 0);
+    
+    // Note that the role is always {@link NamenodeRole#NAMENODE} as this
+    // conversion happens for messages from Namenode to Journal receivers.
+    // Addresses in the registration are unused.
+    return new NamenodeRegistration("", "", storage, NamenodeRole.NAMENODE);
+  }
+
+  /**
+   * Method used for converting {@link JournalInfoProto} sent from Namenode
+   * to Journal receivers to {@link NamenodeRegistration}.
+   */
+  public static JournalInfoProto convertToJournalInfo(
+      NamenodeRegistration reg) {
+    return JournalInfoProto.newBuilder().setClusterID(reg.getClusterID())
+        .setLayoutVersion(reg.getLayoutVersion())
+        .setNamespaceID(reg.getNamespaceID()).build();
+  }
 }

+ 8 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java

@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
-import java.io.DataInput;
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.LinkedList;
@@ -26,13 +24,10 @@ import java.util.List;
 import java.util.Queue;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.DeprecatedUTF8;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.util.LightWeightHashSet;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.WritableUtils;
 
 /**
  * This class extends the DatanodeInfo class with ephemeral information (eg
@@ -158,18 +153,18 @@ public class DatanodeDescriptor extends DatanodeInfo {
    */
   private boolean disallowed = false;
 
-  /** Default constructor */
   public DatanodeDescriptor() {}
   
-  /** DatanodeDescriptor constructor
+  /**
+   * DatanodeDescriptor constructor
    * @param nodeID id of the data node
    */
   public DatanodeDescriptor(DatanodeID nodeID) {
     this(nodeID, 0L, 0L, 0L, 0L, 0, 0);
   }
 
-  /** DatanodeDescriptor constructor
-   * 
+  /**
+   * DatanodeDescriptor constructor
    * @param nodeID id of the data node
    * @param networkLocation location of the data node in network
    */
@@ -178,8 +173,8 @@ public class DatanodeDescriptor extends DatanodeInfo {
     this(nodeID, networkLocation, 0L, 0L, 0L, 0L, 0, 0);
   }
   
-  /** DatanodeDescriptor constructor
-   * 
+  /**
+   * DatanodeDescriptor constructor
    * @param nodeID id of the data node
    * @param capacity capacity of the data node
    * @param dfsUsed space used by the data node
@@ -199,8 +194,8 @@ public class DatanodeDescriptor extends DatanodeInfo {
         failedVolumes);
   }
 
-  /** DatanodeDescriptor constructor
-   * 
+  /**
+   * DatanodeDescriptor constructor
    * @param nodeID id of the data node
    * @param networkLocation location of the data node in network
    * @param capacity capacity of the data node, including space used by non-dfs

+ 47 - 12
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java

@@ -71,6 +71,8 @@ import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.HostsFileReader;
 import org.apache.hadoop.util.ReflectionUtils;
 
+import com.google.common.net.InetAddresses;
+
 /**
  * Manage datanodes, include decommission and other activities.
  */
@@ -353,14 +355,9 @@ public class DatanodeManager {
   private void resolveNetworkLocation (DatanodeDescriptor node) {
     List<String> names = new ArrayList<String>(1);
     if (dnsToSwitchMapping instanceof CachedDNSToSwitchMapping) {
-      // get the node's IP address
       names.add(node.getIpAddr());
     } else {
-      // get the node's host name
-      String hostName = node.getHostName();
-      int colon = hostName.indexOf(":");
-      hostName = (colon==-1)?hostName:hostName.substring(0,colon);
-      names.add(hostName);
+      names.add(node.getHostName());
     }
     
     // resolve its network location
@@ -771,6 +768,40 @@ public class DatanodeManager {
     }
   }
 
+  /**
+   * Parse a DatanodeID from a hosts file entry
+   * @param hostLine of form [hostname|ip][:port]?
+   * @return DatanodeID constructed from the given string
+   */
+  private DatanodeID parseDNFromHostsEntry(String hostLine) {
+    DatanodeID dnId;
+    String hostStr;
+    int port;
+    int idx = hostLine.indexOf(':');
+
+    if (-1 == idx) {
+      hostStr = hostLine;
+      port = DFSConfigKeys.DFS_DATANODE_DEFAULT_PORT;
+    } else {
+      hostStr = hostLine.substring(0, idx);
+      port = Integer.valueOf(hostLine.substring(idx));
+    }
+
+    if (InetAddresses.isInetAddress(hostStr)) {
+      // The IP:port is sufficient for listing in a report
+      dnId = new DatanodeID(hostStr, "", port);
+    } else {
+      String ipAddr = "";
+      try {
+        ipAddr = InetAddress.getByName(hostStr).getHostAddress();
+      } catch (UnknownHostException e) {
+        LOG.warn("Invalid hostname " + hostStr + " in hosts file");
+      }
+      dnId = new DatanodeID(ipAddr, hostStr, port);
+    }
+    return dnId;
+  }
+
   /** For generating datanode reports */
   public List<DatanodeDescriptor> getDatanodeListForReport(
       final DatanodeReportType type) {
@@ -782,7 +813,7 @@ public class DatanodeManager {
     HashMap<String, String> mustList = new HashMap<String, String>();
 
     if (listDeadNodes) {
-      //first load all the nodes listed in include and exclude files.
+      // Put all nodes referenced in the hosts files in the map
       Iterator<String> it = hostsReader.getHosts().iterator();
       while (it.hasNext()) {
         mustList.put(it.next(), "");
@@ -805,7 +836,7 @@ public class DatanodeManager {
         if ( (isDead && listDeadNodes) || (!isDead && listLiveNodes) ) {
           nodes.add(dn);
         }
-        //Remove any form of the this datanode in include/exclude lists.
+        // Remove any nodes we know about from the map
         try {
           InetAddress inet = InetAddress.getByName(dn.getIpAddr());
           // compare hostname(:port)
@@ -814,7 +845,7 @@ public class DatanodeManager {
           // compare ipaddress(:port)
           mustList.remove(inet.getHostAddress().toString());
           mustList.remove(inet.getHostAddress().toString()+ ":" +dn.getXferPort());
-        } catch ( UnknownHostException e ) {
+        } catch (UnknownHostException e) {
           mustList.remove(dn.getName());
           mustList.remove(dn.getIpAddr());
           LOG.warn(e);
@@ -825,9 +856,13 @@ public class DatanodeManager {
     if (listDeadNodes) {
       Iterator<String> it = mustList.keySet().iterator();
       while (it.hasNext()) {
-        DatanodeDescriptor dn = 
-            new DatanodeDescriptor(new DatanodeID(it.next()));
-        dn.setLastUpdate(0);
+        // The remaining nodes are ones that are referenced by the hosts
+        // files but that we do not know about, ie that we have never
+        // head from. Eg. a host that is no longer part of the cluster
+        // or a bogus entry was given in the hosts files
+        DatanodeID dnId = parseDNFromHostsEntry(it.next());
+        DatanodeDescriptor dn = new DatanodeDescriptor(dnId); 
+        dn.setLastUpdate(0); // Consider this node dead for reporting
         nodes.add(dn);
       }
     }

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java

@@ -53,7 +53,8 @@ public final class HdfsServerConstants {
     ROLLBACK("-rollback"),
     FINALIZE("-finalize"),
     IMPORT  ("-importCheckpoint"),
-    BOOTSTRAPSTANDBY("-bootstrapStandby");
+    BOOTSTRAPSTANDBY("-bootstrapStandby"),
+    INITIALIZESHAREDEDITS("-initializeSharedEdits");
     
     private String name = null;
     

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

@@ -668,8 +668,7 @@ public class DataNode extends Configured
    */
   DatanodeRegistration createBPRegistration(NamespaceInfo nsInfo) {
     final String xferIp = streamingAddr.getAddress().getHostAddress();
-    DatanodeRegistration bpRegistration = new DatanodeRegistration(xferIp);
-    bpRegistration.setXferPort(getXferPort());
+    DatanodeRegistration bpRegistration = new DatanodeRegistration(xferIp, getXferPort());
     bpRegistration.setInfoPort(getInfoPort());
     bpRegistration.setIpcPort(getIpcPort());
     bpRegistration.setHostName(hostName);

+ 64 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/journalservice/JournalListener.java

@@ -0,0 +1,64 @@
+/**
+ * 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.journalservice;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.server.protocol.JournalProtocol;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+
+/**
+ * JournalListener is a callback interface to handle journal records
+ * received from the namenode.
+ */
+public interface JournalListener {
+  /**
+   * Check the namespace information returned by a namenode
+   * @param service service that is making the callback
+   * @param info returned namespace information from the namenode
+   * 
+   * The application using {@link JournalService} can stop the service if
+   * {@code info} validation fails.
+   */
+  public void verifyVersion(JournalService service, NamespaceInfo info);
+  
+  /**
+   * Process the received Journal record
+   * @param service {@link JournalService} making the callback
+   * @param firstTxnId first transaction Id in the journal
+   * @param numTxns number of records
+   * @param records journal records
+   * @throws IOException on error
+   * 
+   * Any IOException thrown from the listener is thrown back in 
+   * {@link JournalProtocol#journal}
+   */
+  public void journal(JournalService service, long firstTxnId, int numTxns,
+      byte[] records) throws IOException;
+  
+  /**
+   * Roll the editlog
+   * @param service {@link JournalService} making the callback
+   * @param txid transaction ID to roll at
+   * 
+   * Any IOException thrown from the listener is thrown back in 
+   * {@link JournalProtocol#startLogSegment}
+   */
+  public void rollLogs(JournalService service, long txid) throws IOException;
+}

+ 97 - 128
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalService.java → hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/journalservice/JournalService.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs.server.namenode;
+package org.apache.hadoop.hdfs.server.journalservice;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
@@ -23,15 +23,14 @@ import java.net.InetSocketAddress;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.NameNodeProxies;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
 import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalProtocolService;
 import org.apache.hadoop.hdfs.protocolPB.JournalProtocolPB;
 import org.apache.hadoop.hdfs.protocolPB.JournalProtocolServerSideTranslatorPB;
-import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
+import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.protocol.JournalProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
@@ -60,87 +59,90 @@ import com.google.protobuf.BlockingService;
  */
 public class JournalService implements JournalProtocol {
   public static final Log LOG = LogFactory.getLog(JournalService.class.getName());
+
   private final JournalListener listener;
-  private final boolean internalRpcServer;
   private final InetSocketAddress nnAddress;
   private final NamenodeRegistration registration;
   private final NamenodeProtocol namenode;
-  private volatile State state = State.INIT;
-  private RPC.Server rpcServer;
+  private final StateHandler stateHandler = new StateHandler();
+  private final RPC.Server rpcServer;
   
   enum State {
-    INIT,
-    STARTING_UP,
-    RUNNING,
-    STOPPED;
-  }
-  
-  /**
-   * JournalListener is a callback interface to handle journal records
-   * received from the namenode.
-   */
-  public interface JournalListener {
+    /** The service is initialized and ready to start. */
+    INIT(false, false),
     /**
-     * Check the namespace information returned by a namenode
-     * @param service service that is making the callback
-     * @param info returned namespace information from the namenode
-     * 
-     * The application using {@link JournalService} can stop the service if
-     * {@code info} validation fails.
+     * RPC server is started.
+     * The service is ready to receive requests from namenode.
      */
-    public void verifyVersion(JournalService service, NamespaceInfo info);
-    
+    STARTED(false, false),
+    /** The service is fenced by a namenode and waiting for roll. */
+    WAITING_FOR_ROLL(false, true),
     /**
-     * Process the received Journal record
-     * @param service {@link JournalService} making the callback
-     * @param firstTxnId first transaction Id in the journal
-     * @param numTxns number of records
-     * @param records journal records
-     * @throws IOException on error
-     * 
-     * Any IOException thrown from the listener is thrown back in 
-     * {@link JournalProtocol#journal}
+     * The existing log is syncing with another source
+     * and it accepts journal from Namenode.
      */
-    public void journal(JournalService service, long firstTxnId, int numTxns,
-        byte[] records) throws IOException;
+    SYNCING(true, true),
+    /** The existing log is in sync and it accepts journal from Namenode. */
+    IN_SYNC(true, true),
+    /** The service is stopped. */
+    STOPPED(false, false);
+
+    final boolean isJournalAllowed;
+    final boolean isStartLogSegmentAllowed;
     
-    /**
-     * Roll the editlog
-     * @param service {@link JournalService} making the callback
-     * @param txid transaction ID to roll at
-     * 
-     * Any IOException thrown from the listener is thrown back in 
-     * {@link JournalProtocol#startLogSegment}
-     */
-    public void rollLogs(JournalService service, long txid) throws IOException;
+    State(boolean isJournalAllowed, boolean isStartLogSegmentAllowed) {
+      this.isJournalAllowed = isJournalAllowed;
+      this.isStartLogSegmentAllowed = isStartLogSegmentAllowed;
+    }
   }
   
-  /**
-   * Constructor to create {@link JournalService} based on an existing RPC server.
-   * After creating the service, the caller needs to start the RPC server.
-   * 
-   * @param conf Configuration
-   * @param nnAddr host:port for the active Namenode's RPC server
-   * @param listener call-back interface to listen to journal activities
-   * @param rpcServer RPC server if the application has already one, which can be
-   *          reused. If this is null, then the RPC server is started by
-   *          {@link JournalService}
-   * @param reg namenode registration information if there is one already, say
-   *          if you are using this service in namenode. If it is null, then the
-   *          service creates a new registration.
-   * @throws IOException on error
-   */
-  JournalService(Configuration conf, InetSocketAddress nnAddr,
-      JournalListener listener, RPC.Server rpcServer, NamenodeRegistration reg)
-      throws IOException {
-    this.nnAddress = nnAddr;
-    this.listener = listener;
-    this.registration = reg;
-    this.internalRpcServer = false;
-    this.namenode = NameNodeProxies.createNonHAProxy(conf, nnAddr,
-        NamenodeProtocol.class, UserGroupInformation.getCurrentUser(), true)
-        .getProxy();
-    initRpcServer(conf, rpcServer);
+  static class StateHandler {
+    State current = State.INIT;
+    
+    synchronized void start() {
+      if (current != State.INIT) {
+        throw new IllegalStateException("Service cannot be started in "
+            + current + " state.");
+      }
+      current = State.STARTED;
+    }
+
+    synchronized void waitForRoll() {
+      if (current != State.STARTED) {
+        throw new IllegalStateException("Cannot wait-for-roll in " + current
+            + " state.");
+      }
+      current = State.WAITING_FOR_ROLL;
+    }
+
+    synchronized void startLogSegment() throws IOException {
+      if (current == State.WAITING_FOR_ROLL) {
+        current = State.SYNCING;
+      }
+    }
+
+    synchronized void isStartLogSegmentAllowed() throws IOException {
+      if (!current.isStartLogSegmentAllowed) {
+        throw new IOException("Cannot start log segment in " + current
+            + " state.");
+      }
+    }
+
+    synchronized void isJournalAllowed() throws IOException {
+      if (!current.isJournalAllowed) {
+        throw new IOException("Cannot journal in " + current + " state.");
+      }
+    }
+
+    synchronized boolean isStopped() {
+      if (current == State.STOPPED) {
+        LOG.warn("Ignore stop request since the service is in " + current
+            + " state.");
+        return true;
+      }
+      current = State.STOPPED;
+      return false;
+    }
   }
   
   /**
@@ -160,11 +162,11 @@ public class JournalService implements JournalProtocol {
       throws IOException {
     this.nnAddress = nnAddr;
     this.listener = listener;
-    this.internalRpcServer = true;
     this.namenode = NameNodeProxies.createNonHAProxy(conf, nnAddr,
         NamenodeProtocol.class, UserGroupInformation.getCurrentUser(), true)
         .getProxy();
-    initRpcServer(conf, serverAddress);
+    this.rpcServer = createRpcServer(conf, serverAddress, this);
+
     String addr = NetUtils.getHostPortString(rpcServer.getListenerAddress());
     StorageInfo storage = new StorageInfo(
         LayoutVersion.getCurrentLayoutVersion(), 0, "", 0);
@@ -176,23 +178,13 @@ public class JournalService implements JournalProtocol {
    * Start the service.
    */
   public void start() {
-    synchronized(this) {
-      if (state != State.INIT) {
-        LOG.info("Service cannot be started in state - " + state);
-        return;
-      }
-      state = State.STARTING_UP;
-    }
+    stateHandler.start();
+
     // Start the RPC server
-    if (internalRpcServer) {
-      LOG.info("Starting rpc server");
-      rpcServer.start();
-    }
-    
-    boolean registered = false;
-    boolean handshakeComplete = false;
-    boolean rollEdits = false;
-    while (state == State.STARTING_UP) {
+    LOG.info("Starting rpc server");
+    rpcServer.start();
+
+    for(boolean registered = false, handshakeComplete = false; ; ) {
       try {
         // Perform handshake
         if (!handshakeComplete) {
@@ -206,12 +198,6 @@ public class JournalService implements JournalProtocol {
           registerWithNamenode();
           registered = true;
           LOG.info("Registration completed");
-        }
-        
-        if (!rollEdits) {
-          namenode.rollEditLog();
-          rollEdits = true;
-          LOG.info("Editlog roll completed");
           break;
         }
       } catch (IOException ioe) {
@@ -226,10 +212,13 @@ public class JournalService implements JournalProtocol {
         LOG.warn("Encountered exception ", ie);
       }
     }
-    synchronized(this) {
-      state = State.RUNNING;
+
+    stateHandler.waitForRoll();
+    try {
+      namenode.rollEditLog();
+    } catch (IOException e) {
+      LOG.warn("Encountered exception ", e);
     }
-    
   }
 
   /**
@@ -237,15 +226,8 @@ public class JournalService implements JournalProtocol {
    * RPC Server must be stopped the application.
    */
   public void stop() {
-    synchronized (this) {
-      if (state == State.STOPPED) {
-        return;
-      }
-      state = State.STOPPED;
-    }
-    if (internalRpcServer && rpcServer != null) {
+    if (!stateHandler.isStopped()) {
       rpcServer.stop();
-      rpcServer = null;
     }
   }
 
@@ -255,6 +237,7 @@ public class JournalService implements JournalProtocol {
     if (LOG.isTraceEnabled()) {
       LOG.trace("Received journal " + firstTxnId + " " + numTxns);
     }
+    stateHandler.isJournalAllowed();
     verify(registration);
     listener.journal(this, firstTxnId, numTxns, records);
   }
@@ -265,37 +248,23 @@ public class JournalService implements JournalProtocol {
     if (LOG.isTraceEnabled()) {
       LOG.trace("Received startLogSegment " + txid);
     }
+    stateHandler.isStartLogSegmentAllowed();
     verify(registration);
     listener.rollLogs(this, txid);
+    stateHandler.startLogSegment();
   }
 
-  /** 
-   * Stand alone mode where RPC Server is created and managed by this service 
-   */
-  private void initRpcServer(Configuration conf, InetSocketAddress serverAddress)
-      throws IOException {
+  /** Create an RPC server. */
+  private static RPC.Server createRpcServer(Configuration conf,
+      InetSocketAddress address, JournalProtocol impl) throws IOException {
     RPC.setProtocolEngine(conf, JournalProtocolPB.class,
         ProtobufRpcEngine.class);
     JournalProtocolServerSideTranslatorPB xlator = 
-        new JournalProtocolServerSideTranslatorPB(this);
-    BlockingService service = 
-        JournalProtocolService.newReflectiveBlockingService(xlator);
-    rpcServer = RPC.getServer(JournalProtocolPB.class, service,
-        serverAddress.getHostName(), serverAddress.getPort(), 1, false, conf,
-        null);
-  }
-
-  /**
-   * RPC Server is created and managed by the application - used by this service
-   */
-  private void initRpcServer(Configuration conf, RPC.Server server)
-      throws IOException {
-    rpcServer = server;
-    JournalProtocolServerSideTranslatorPB xlator = 
-        new JournalProtocolServerSideTranslatorPB(this);
+        new JournalProtocolServerSideTranslatorPB(impl);
     BlockingService service = 
         JournalProtocolService.newReflectiveBlockingService(xlator);
-    DFSUtil.addPBProtocol(conf, JournalProtocolPB.class, service, rpcServer);
+    return RPC.getServer(JournalProtocolPB.class, service,
+        address.getHostName(), address.getPort(), 1, false, conf, null);
   }
   
   private void verify(NamenodeRegistration reg) throws IOException {

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

@@ -331,8 +331,7 @@ public class BackupImage extends FSImage {
    */
   private synchronized void setState(BNState newState) {
     if (LOG.isDebugEnabled()) {
-      LOG.debug("State transition " + bnState + " -> " + newState,
-          new Exception("trace"));
+      LOG.debug("State transition " + bnState + " -> " + newState);
     }
     bnState = newState;
   }

+ 27 - 12
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java

@@ -27,17 +27,15 @@ import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.NameNodeProxies;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalProtocolService;
 import org.apache.hadoop.hdfs.protocolPB.JournalProtocolPB;
 import org.apache.hadoop.hdfs.protocolPB.JournalProtocolServerSideTranslatorPB;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.Storage;
-import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.JournalProtocol;
-import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
@@ -225,8 +223,6 @@ public class BackupNode extends NameNode {
   
   static class BackupNodeRpcServer extends NameNodeRpcServer implements
       JournalProtocol {
-    private final String nnRpcAddress;
-    
     private BackupNodeRpcServer(Configuration conf, BackupNode nn)
         throws IOException {
       super(conf, nn);
@@ -236,8 +232,31 @@ public class BackupNode extends NameNode {
           .newReflectiveBlockingService(journalProtocolTranslator);
       DFSUtil.addPBProtocol(conf, JournalProtocolPB.class, service,
           this.clientRpcServer);
-      nnRpcAddress = nn.nnRpcAddress;
     }
+    
+    /** 
+     * Verifies a journal request
+     * @param nodeReg node registration
+     * @throws UnregisteredNodeException if the registration is invalid
+     */
+    void verifyJournalRequest(NamenodeRegistration reg) throws IOException {
+      verifyVersion(reg.getLayoutVersion());
+      String errorMsg = null;
+      int expectedNamespaceID = namesystem.getNamespaceInfo().getNamespaceID();
+      if (reg.getNamespaceID() != expectedNamespaceID) {
+        errorMsg = "Invalid namespaceID in journal request - expected " + expectedNamespaceID
+            + " actual " + reg.getNamespaceID();
+        LOG.warn(errorMsg);
+        throw new UnregisteredNodeException(reg);
+      } 
+      if (!reg.getClusterID().equals(namesystem.getClusterId())) {
+        errorMsg = "Invalid clusterId in journal request - expected "
+            + reg.getClusterID() + " actual " + namesystem.getClusterId();
+        LOG.warn(errorMsg);
+        throw new UnregisteredNodeException(reg);
+      }
+    }
+
 
     /////////////////////////////////////////////////////
     // BackupNodeProtocol implementation for backup node.
@@ -246,8 +265,7 @@ public class BackupNode extends NameNode {
     public void startLogSegment(NamenodeRegistration registration, long txid)
         throws IOException {
       namesystem.checkOperation(OperationCategory.JOURNAL);
-      verifyRequest(registration);
-      
+      verifyJournalRequest(registration);
       getBNImage().namenodeStartedLogSegment(txid);
     }
     
@@ -256,10 +274,7 @@ public class BackupNode extends NameNode {
         long firstTxId, int numTxns,
         byte[] records) throws IOException {
       namesystem.checkOperation(OperationCategory.JOURNAL);
-      verifyRequest(nnReg);
-      if(!nnRpcAddress.equals(nnReg.getAddress()))
-        throw new IOException("Journal request from unexpected name-node: "
-            + nnReg.getAddress() + " expecting " + nnRpcAddress);
+      verifyJournalRequest(nnReg);
       getBNImage().journal(firstTxId, numTxns, records);
     }
 

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

@@ -33,7 +33,7 @@ import com.google.common.annotations.VisibleForTesting;
  * An implementation of the abstract class {@link EditLogInputStream}, which
  * reads edits from a local file.
  */
-class EditLogFileInputStream extends EditLogInputStream {
+public class EditLogFileInputStream extends EditLogInputStream {
   private final File file;
   private final FileInputStream fStream;
   final private long firstTxId;
@@ -67,7 +67,7 @@ class EditLogFileInputStream extends EditLogInputStream {
    * @throws IOException if an actual IO error occurs while reading the
    *         header
    */
-  EditLogFileInputStream(File name, long firstTxId, long lastTxId,
+  public EditLogFileInputStream(File name, long firstTxId, long lastTxId,
       boolean isInProgress)
       throws LogHeaderCorruptException, IOException {
     file = name;

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

@@ -64,7 +64,7 @@ public class EditLogFileOutputStream extends EditLogOutputStream {
    *          Size of flush buffer
    * @throws IOException
    */
-  EditLogFileOutputStream(File name, int size) throws IOException {
+  public EditLogFileOutputStream(File name, int size) throws IOException {
     super();
     file = name;
     doubleBuf = new EditsDoubleBuffer(size);
@@ -174,7 +174,7 @@ public class EditLogFileOutputStream extends EditLogOutputStream {
    * accumulates new log records while readyBuffer will be flushed and synced.
    */
   @Override
-  protected void flushAndSync() throws IOException {
+  public void flushAndSync() throws IOException {
     if (fp == null) {
       throw new IOException("Trying to use aborted output stream");
     }

+ 580 - 24
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java

@@ -21,7 +21,10 @@ import java.util.zip.CheckedInputStream;
 import java.util.zip.Checksum;
 import java.util.Arrays;
 import java.util.EnumMap;
+import java.util.List;
 
+import org.apache.commons.codec.DecoderException;
+import org.apache.commons.codec.binary.Hex;
 import org.apache.hadoop.fs.ChecksumException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -39,10 +42,17 @@ import org.apache.hadoop.security.token.delegation.DelegationKey;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.ArrayWritable;
+import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableFactories;
 import org.apache.hadoop.io.WritableFactory;
+import org.apache.hadoop.hdfs.util.XMLUtils;
+import org.apache.hadoop.hdfs.util.XMLUtils.InvalidXmlException;
+import org.apache.hadoop.hdfs.util.XMLUtils.Stanza;
 import org.apache.hadoop.hdfs.DeprecatedUTF8;
+import org.xml.sax.ContentHandler;
+import org.xml.sax.SAXException;
+import org.xml.sax.helpers.AttributesImpl;
 
 import java.io.DataInput;
 import java.io.DataOutput;
@@ -59,7 +69,7 @@ import java.io.EOFException;
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
 public abstract class FSEditLogOp {
-  final FSEditLogOpCodes opCode;
+  public final FSEditLogOpCodes opCode;
   long txid;
 
 
@@ -121,7 +131,7 @@ public abstract class FSEditLogOp {
   abstract void readFields(DataInputStream in, int logVersion)
       throws IOException;
 
-  abstract void writeFields(DataOutputStream out)
+  public abstract void writeFields(DataOutputStream out)
       throws IOException;
 
   static interface BlockListUpdatingOp {
@@ -201,7 +211,8 @@ public abstract class FSEditLogOp {
       return (T)this;
     }
 
-    @Override 
+    @Override
+    public 
     void writeFields(DataOutputStream out) throws IOException {
       FSImageSerialization.writeString(path, out);
       FSImageSerialization.writeShort(replication, out);
@@ -312,6 +323,49 @@ public abstract class FSEditLogOp {
       builder.append("]");
       return builder.toString();
     }
+    
+    @Override
+    protected void toXml(ContentHandler contentHandler) throws SAXException {
+      XMLUtils.addSaxString(contentHandler, "LENGTH",
+          Integer.valueOf(length).toString());
+      XMLUtils.addSaxString(contentHandler, "PATH", path);
+      XMLUtils.addSaxString(contentHandler, "REPLICATION",
+          Short.valueOf(replication).toString());
+      XMLUtils.addSaxString(contentHandler, "MTIME",
+          Long.valueOf(mtime).toString());
+      XMLUtils.addSaxString(contentHandler, "ATIME",
+          Long.valueOf(atime).toString());
+      XMLUtils.addSaxString(contentHandler, "BLOCKSIZE",
+          Long.valueOf(blockSize).toString());
+      XMLUtils.addSaxString(contentHandler, "CLIENT_NAME", clientName);
+      XMLUtils.addSaxString(contentHandler, "CLIENT_MACHINE", clientMachine);
+      for (Block b : blocks) {
+        FSEditLogOp.blockToXml(contentHandler, b);
+      }
+      FSEditLogOp.permissionStatusToXml(contentHandler, permissions);
+    }
+
+    @Override void fromXml(Stanza st) throws InvalidXmlException {
+      this.length = Integer.valueOf(st.getValue("LENGTH"));
+      this.path = st.getValue("PATH");
+      this.replication = Short.valueOf(st.getValue("REPLICATION"));
+      this.mtime = Long.valueOf(st.getValue("MTIME"));
+      this.atime = Long.valueOf(st.getValue("ATIME"));
+      this.blockSize = Long.valueOf(st.getValue("BLOCKSIZE"));
+      this.clientName = st.getValue("CLIENT_NAME");
+      this.clientMachine = st.getValue("CLIENT_MACHINE");
+      if (st.hasChildren("BLOCK")) {
+        List<Stanza> blocks = st.getChildren("BLOCK");
+        this.blocks = new Block[blocks.size()];
+        for (int i = 0; i < blocks.size(); i++) {
+          this.blocks[i] = FSEditLogOp.blockFromXml(blocks.get(i));
+        }
+      } else {
+        this.blocks = new Block[0];
+      }
+      this.permissions =
+          permissionStatusFromXml(st.getChildren("PERMISSION_STATUS").get(0));
+    }
   }
 
   static class AddOp extends AddCloseOp {
@@ -391,6 +445,7 @@ public abstract class FSEditLogOp {
     }
 
     @Override
+    public
     void writeFields(DataOutputStream out) throws IOException {
       FSImageSerialization.writeString(path, out);
       FSImageSerialization.writeCompactBlockArray(blocks, out);
@@ -418,6 +473,23 @@ public abstract class FSEditLogOp {
         .append("]");
       return sb.toString();
     }
+    
+    @Override
+    protected void toXml(ContentHandler contentHandler) throws SAXException {
+      XMLUtils.addSaxString(contentHandler, "PATH", path);
+      for (Block b : blocks) {
+        FSEditLogOp.blockToXml(contentHandler, b);
+      }
+    }
+    
+    @Override void fromXml(Stanza st) throws InvalidXmlException {
+      this.path = st.getValue("PATH");
+      List<Stanza> blocks = st.getChildren("BLOCK");
+      this.blocks = new Block[blocks.size()];
+      for (int i = 0; i < blocks.size(); i++) {
+        this.blocks[i] = FSEditLogOp.blockFromXml(blocks.get(i));
+      }
+    }
   }
 
   static class SetReplicationOp extends FSEditLogOp {
@@ -443,7 +515,8 @@ public abstract class FSEditLogOp {
       return this;
     }
 
-    @Override 
+    @Override
+    public 
     void writeFields(DataOutputStream out) throws IOException {
       FSImageSerialization.writeString(path, out);
       FSImageSerialization.writeShort(replication, out);
@@ -474,6 +547,18 @@ public abstract class FSEditLogOp {
       builder.append("]");
       return builder.toString();
     }
+    
+    @Override
+    protected void toXml(ContentHandler contentHandler) throws SAXException {
+      XMLUtils.addSaxString(contentHandler, "PATH", path);
+      XMLUtils.addSaxString(contentHandler, "REPLICATION",
+          Short.valueOf(replication).toString());
+    }
+    
+    @Override void fromXml(Stanza st) throws InvalidXmlException {
+      this.path = st.getValue("PATH");
+      this.replication = Short.valueOf(st.getValue("REPLICATION"));
+    }
   }
 
   static class ConcatDeleteOp extends FSEditLogOp {
@@ -506,7 +591,8 @@ public abstract class FSEditLogOp {
       return this;
     }
 
-    @Override 
+    @Override
+    public 
     void writeFields(DataOutputStream out) throws IOException {
       FSImageSerialization.writeString(trg, out);
             
@@ -567,6 +653,38 @@ public abstract class FSEditLogOp {
       builder.append("]");
       return builder.toString();
     }
+    
+    @Override
+    protected void toXml(ContentHandler contentHandler) throws SAXException {
+      XMLUtils.addSaxString(contentHandler, "LENGTH",
+          Integer.valueOf(length).toString());
+      XMLUtils.addSaxString(contentHandler, "TRG", trg);
+      XMLUtils.addSaxString(contentHandler, "TIMESTAMP",
+          Long.valueOf(timestamp).toString());
+      contentHandler.startElement("", "", "SOURCES", new AttributesImpl());
+      for (int i = 0; i < srcs.length; ++i) {
+        XMLUtils.addSaxString(contentHandler,
+            "SOURCE" + (i + 1), srcs[i]);
+      }
+      contentHandler.endElement("", "", "SOURCES");
+    }
+    
+    @Override void fromXml(Stanza st) throws InvalidXmlException {
+      this.length = Integer.valueOf(st.getValue("LENGTH"));
+      this.trg = st.getValue("TRG");
+      this.timestamp = Long.valueOf(st.getValue("TIMESTAMP"));
+      List<Stanza> sources = st.getChildren("SOURCES");
+      int i = 0;
+      while (true) {
+        if (!sources.get(0).hasChildren("SOURCE" + (i + 1)))
+          break;
+        i++;
+      }
+      srcs = new String[i];
+      for (i = 0; i < srcs.length; i++) {
+        srcs[i] = sources.get(0).getValue("SOURCE" + (i + 1));
+      }
+    }
   }
 
   static class RenameOldOp extends FSEditLogOp {
@@ -599,7 +717,8 @@ public abstract class FSEditLogOp {
       return this;
     }
 
-    @Override 
+    @Override
+    public 
     void writeFields(DataOutputStream out) throws IOException {
       FSImageSerialization.writeString(src, out);
       FSImageSerialization.writeString(dst, out);
@@ -643,6 +762,23 @@ public abstract class FSEditLogOp {
       builder.append("]");
       return builder.toString();
     }
+    
+    @Override
+    protected void toXml(ContentHandler contentHandler) throws SAXException {
+      XMLUtils.addSaxString(contentHandler, "LENGTH",
+          Integer.valueOf(length).toString());
+      XMLUtils.addSaxString(contentHandler, "SRC", src);
+      XMLUtils.addSaxString(contentHandler, "DST", dst);
+      XMLUtils.addSaxString(contentHandler, "TIMESTAMP",
+          Long.valueOf(timestamp).toString());
+    }
+    
+    @Override void fromXml(Stanza st) throws InvalidXmlException {
+      this.length = Integer.valueOf(st.getValue("LENGTH"));
+      this.src = st.getValue("SRC");
+      this.dst = st.getValue("DST");
+      this.timestamp = Long.valueOf(st.getValue("TIMESTAMP"));
+    }
   }
 
   static class DeleteOp extends FSEditLogOp {
@@ -669,7 +805,8 @@ public abstract class FSEditLogOp {
       return this;
     }
 
-    @Override 
+    @Override
+    public 
     void writeFields(DataOutputStream out) throws IOException {
       FSImageSerialization.writeString(path, out);
       FSImageSerialization.writeLong(timestamp, out);
@@ -708,8 +845,23 @@ public abstract class FSEditLogOp {
       builder.append("]");
       return builder.toString();
     }
+    
+    @Override
+    protected void toXml(ContentHandler contentHandler) throws SAXException {
+      XMLUtils.addSaxString(contentHandler, "LENGTH",
+          Integer.valueOf(length).toString());
+      XMLUtils.addSaxString(contentHandler, "PATH", path);
+      XMLUtils.addSaxString(contentHandler, "TIMESTAMP",
+          Long.valueOf(timestamp).toString());
+    }
+    
+    @Override void fromXml(Stanza st) throws InvalidXmlException {
+      this.length = Integer.valueOf(st.getValue("LENGTH"));
+      this.path = st.getValue("PATH");
+      this.timestamp = Long.valueOf(st.getValue("TIMESTAMP"));
+    }
   }
-
+    
   static class MkdirOp extends FSEditLogOp {
     int length;
     String path;
@@ -740,7 +892,8 @@ public abstract class FSEditLogOp {
       return this;
     }
 
-    @Override 
+    @Override
+    public 
     void writeFields(DataOutputStream out) throws IOException {
       FSImageSerialization.writeString(path, out);
       FSImageSerialization.writeLong(timestamp, out); // mtime
@@ -797,6 +950,24 @@ public abstract class FSEditLogOp {
       builder.append("]");
       return builder.toString();
     }
+
+    @Override
+    protected void toXml(ContentHandler contentHandler) throws SAXException {
+      XMLUtils.addSaxString(contentHandler, "LENGTH",
+          Integer.valueOf(length).toString());
+      XMLUtils.addSaxString(contentHandler, "PATH", path);
+      XMLUtils.addSaxString(contentHandler, "TIMESTAMP",
+          Long.valueOf(timestamp).toString());
+      FSEditLogOp.permissionStatusToXml(contentHandler, permissions);
+    }
+    
+    @Override void fromXml(Stanza st) throws InvalidXmlException {
+      this.length = Integer.valueOf(st.getValue("LENGTH"));
+      this.path = st.getValue("PATH");
+      this.timestamp = Long.valueOf(st.getValue("TIMESTAMP"));
+      this.permissions =
+          permissionStatusFromXml(st.getChildren("PERMISSION_STATUS").get(0));
+    }
   }
 
   static class SetGenstampOp extends FSEditLogOp {
@@ -816,7 +987,8 @@ public abstract class FSEditLogOp {
       return this;
     }
     
-    @Override 
+    @Override
+    public 
     void writeFields(DataOutputStream out) throws IOException {
       FSImageSerialization.writeLong(genStamp, out);
     }
@@ -839,6 +1011,16 @@ public abstract class FSEditLogOp {
       builder.append("]");
       return builder.toString();
     }
+    
+    @Override
+    protected void toXml(ContentHandler contentHandler) throws SAXException {
+      XMLUtils.addSaxString(contentHandler, "GENSTAMP",
+          Long.valueOf(genStamp).toString());
+    }
+    
+    @Override void fromXml(Stanza st) throws InvalidXmlException {
+      this.genStamp = Long.valueOf(st.getValue("GENSTAMP"));
+    }
   }
 
   static class SetPermissionsOp extends FSEditLogOp {
@@ -864,7 +1046,8 @@ public abstract class FSEditLogOp {
       return this;
     }
 
-    @Override 
+    @Override
+    public 
     void writeFields(DataOutputStream out) throws IOException {
       FSImageSerialization.writeString(src, out);
       permissions.write(out);
@@ -891,6 +1074,19 @@ public abstract class FSEditLogOp {
       builder.append("]");
       return builder.toString();
     }
+    
+    @Override
+    protected void toXml(ContentHandler contentHandler) throws SAXException {
+      XMLUtils.addSaxString(contentHandler, "SRC", src);
+      XMLUtils.addSaxString(contentHandler, "MODE",
+          Short.valueOf(permissions.toShort()).toString());
+    }
+    
+    @Override void fromXml(Stanza st) throws InvalidXmlException {
+      this.src = st.getValue("SRC");
+      this.permissions = new FsPermission(
+          Short.valueOf(st.getValue("MODE")));
+    }
   }
 
   static class SetOwnerOp extends FSEditLogOp {
@@ -922,7 +1118,8 @@ public abstract class FSEditLogOp {
       return this;
     }
 
-    @Override 
+    @Override
+    public 
     void writeFields(DataOutputStream out) throws IOException {
       FSImageSerialization.writeString(src, out);
       FSImageSerialization.writeString(username == null ? "" : username, out);
@@ -953,6 +1150,25 @@ public abstract class FSEditLogOp {
       builder.append("]");
       return builder.toString();
     }
+    
+    @Override
+    protected void toXml(ContentHandler contentHandler) throws SAXException {
+      XMLUtils.addSaxString(contentHandler, "SRC", src);
+      XMLUtils.addSaxString(contentHandler, "USERNAME", username);
+      if (groupname != null) {
+        XMLUtils.addSaxString(contentHandler, "GROUPNAME", groupname);
+      }
+    }
+    
+    @Override void fromXml(Stanza st) throws InvalidXmlException {
+      this.src = st.getValue("SRC");
+      this.username = st.getValue("USERNAME");
+      if (st.hasChildren("GROUPNAME")) {
+        this.groupname = st.getValue("GROUPNAME");
+      } else {
+        this.groupname = null;
+      }
+    }
   }
 
   static class SetNSQuotaOp extends FSEditLogOp {
@@ -968,7 +1184,8 @@ public abstract class FSEditLogOp {
         .get(OP_SET_NS_QUOTA);
     }
 
-    @Override 
+    @Override
+    public 
     void writeFields(DataOutputStream out) throws IOException {
       throw new IOException("Deprecated");      
     }
@@ -994,6 +1211,18 @@ public abstract class FSEditLogOp {
       builder.append("]");
       return builder.toString();
     }
+    
+    @Override
+    protected void toXml(ContentHandler contentHandler) throws SAXException {
+      XMLUtils.addSaxString(contentHandler, "SRC", src);
+      XMLUtils.addSaxString(contentHandler, "NSQUOTA",
+          Long.valueOf(nsQuota).toString());
+    }
+    
+    @Override void fromXml(Stanza st) throws InvalidXmlException {
+      this.src = st.getValue("SRC");
+      this.nsQuota = Long.valueOf(st.getValue("NSQUOTA"));
+    }
   }
 
   static class ClearNSQuotaOp extends FSEditLogOp {
@@ -1008,7 +1237,8 @@ public abstract class FSEditLogOp {
         .get(OP_CLEAR_NS_QUOTA);
     }
 
-    @Override 
+    @Override
+    public 
     void writeFields(DataOutputStream out) throws IOException {
       throw new IOException("Deprecated");      
     }
@@ -1031,6 +1261,15 @@ public abstract class FSEditLogOp {
       builder.append("]");
       return builder.toString();
     }
+    
+    @Override
+    protected void toXml(ContentHandler contentHandler) throws SAXException {
+      XMLUtils.addSaxString(contentHandler, "SRC", src);
+    }
+    
+    @Override void fromXml(Stanza st) throws InvalidXmlException {
+      this.src = st.getValue("SRC");
+    }
   }
 
   static class SetQuotaOp extends FSEditLogOp {
@@ -1062,7 +1301,8 @@ public abstract class FSEditLogOp {
       return this;
     }
 
-    @Override 
+    @Override
+    public 
     void writeFields(DataOutputStream out) throws IOException {
       FSImageSerialization.writeString(src, out);
       FSImageSerialization.writeLong(nsQuota, out);
@@ -1093,6 +1333,21 @@ public abstract class FSEditLogOp {
       builder.append("]");
       return builder.toString();
     }
+    
+    @Override
+    protected void toXml(ContentHandler contentHandler) throws SAXException {
+      XMLUtils.addSaxString(contentHandler, "SRC", src);
+      XMLUtils.addSaxString(contentHandler, "NSQUOTA",
+          Long.valueOf(nsQuota).toString());
+      XMLUtils.addSaxString(contentHandler, "DSQUOTA",
+          Long.valueOf(dsQuota).toString());
+    }
+    
+    @Override void fromXml(Stanza st) throws InvalidXmlException {
+      this.src = st.getValue("SRC");
+      this.nsQuota = Long.valueOf(st.getValue("NSQUOTA"));
+      this.dsQuota = Long.valueOf(st.getValue("DSQUOTA"));
+    }
   }
 
   static class TimesOp extends FSEditLogOp {
@@ -1125,7 +1380,8 @@ public abstract class FSEditLogOp {
       return this;
     }
 
-    @Override 
+    @Override
+    public 
     void writeFields(DataOutputStream out) throws IOException {
       FSImageSerialization.writeString(path, out);
       FSImageSerialization.writeLong(mtime, out);
@@ -1170,6 +1426,24 @@ public abstract class FSEditLogOp {
       builder.append("]");
       return builder.toString();
     }
+    
+    @Override
+    protected void toXml(ContentHandler contentHandler) throws SAXException {
+      XMLUtils.addSaxString(contentHandler, "LENGTH",
+          Integer.valueOf(length).toString());
+      XMLUtils.addSaxString(contentHandler, "PATH", path);
+      XMLUtils.addSaxString(contentHandler, "MTIME",
+          Long.valueOf(mtime).toString());
+      XMLUtils.addSaxString(contentHandler, "ATIME",
+          Long.valueOf(atime).toString());
+    }
+    
+    @Override void fromXml(Stanza st) throws InvalidXmlException {
+      this.length = Integer.valueOf(st.getValue("LENGTH"));
+      this.path = st.getValue("PATH");
+      this.mtime = Long.valueOf(st.getValue("MTIME"));
+      this.atime = Long.valueOf(st.getValue("ATIME"));
+    }
   }
 
   static class SymlinkOp extends FSEditLogOp {
@@ -1214,7 +1488,8 @@ public abstract class FSEditLogOp {
       return this;
     }
 
-    @Override 
+    @Override
+    public 
     void writeFields(DataOutputStream out) throws IOException {
       FSImageSerialization.writeString(path, out);
       FSImageSerialization.writeString(value, out);
@@ -1268,6 +1543,29 @@ public abstract class FSEditLogOp {
       builder.append("]");
       return builder.toString();
     }
+    
+    @Override
+    protected void toXml(ContentHandler contentHandler) throws SAXException {
+      XMLUtils.addSaxString(contentHandler, "LENGTH",
+          Integer.valueOf(length).toString());
+      XMLUtils.addSaxString(contentHandler, "PATH", path);
+      XMLUtils.addSaxString(contentHandler, "VALUE", value);
+      XMLUtils.addSaxString(contentHandler, "MTIME",
+          Long.valueOf(mtime).toString());
+      XMLUtils.addSaxString(contentHandler, "ATIME",
+          Long.valueOf(atime).toString());
+      FSEditLogOp.permissionStatusToXml(contentHandler, permissionStatus);
+    }
+
+    @Override void fromXml(Stanza st) throws InvalidXmlException {
+      this.length = Integer.valueOf(st.getValue("LENGTH"));
+      this.path = st.getValue("PATH");
+      this.value = st.getValue("VALUE");
+      this.mtime = Long.valueOf(st.getValue("MTIME"));
+      this.atime = Long.valueOf(st.getValue("ATIME"));
+      this.permissionStatus =
+          permissionStatusFromXml(st.getChildren("PERMISSION_STATUS").get(0));
+    }
   }
 
   static class RenameOp extends FSEditLogOp {
@@ -1306,7 +1604,8 @@ public abstract class FSEditLogOp {
       return this;
     }
 
-    @Override 
+    @Override
+    public 
     void writeFields(DataOutputStream out) throws IOException {
       FSImageSerialization.writeString(src, out);
       FSImageSerialization.writeString(dst, out);
@@ -1375,6 +1674,44 @@ public abstract class FSEditLogOp {
       builder.append("]");
       return builder.toString();
     }
+    
+    @Override
+    protected void toXml(ContentHandler contentHandler) throws SAXException {
+      XMLUtils.addSaxString(contentHandler, "LENGTH",
+          Integer.valueOf(length).toString());
+      XMLUtils.addSaxString(contentHandler, "SRC", src);
+      XMLUtils.addSaxString(contentHandler, "DST", dst);
+      XMLUtils.addSaxString(contentHandler, "TIMESTAMP",
+          Long.valueOf(timestamp).toString());
+      StringBuilder bld = new StringBuilder();
+      String prefix = "";
+      for (Rename r : options) {
+        bld.append(prefix).append(r.toString());
+        prefix = "|";
+      }
+      XMLUtils.addSaxString(contentHandler, "OPTIONS", bld.toString());
+    }
+    
+    @Override void fromXml(Stanza st) throws InvalidXmlException {
+      this.length = Integer.valueOf(st.getValue("LENGTH"));
+      this.src = st.getValue("SRC");
+      this.dst = st.getValue("DST");
+      this.timestamp = Long.valueOf(st.getValue("TIMESTAMP"));
+      String opts = st.getValue("OPTIONS");
+      String o[] = opts.split("\\|");
+      this.options = new Rename[o.length];
+      for (int i = 0; i < o.length; i++) {
+        if (o[i].equals(""))
+          continue;
+        try {
+          this.options[i] = Rename.valueOf(o[i]);
+        } finally {
+          if (this.options[i] == null) {
+            System.err.println("error parsing Rename value: \"" + o[i] + "\"");
+          }
+        }
+      }
+    }
   }
 
   static class ReassignLeaseOp extends FSEditLogOp {
@@ -1406,7 +1743,8 @@ public abstract class FSEditLogOp {
       return this;
     }
 
-    @Override 
+    @Override
+    public 
     void writeFields(DataOutputStream out) throws IOException {
       FSImageSerialization.writeString(leaseHolder, out);
       FSImageSerialization.writeString(path, out);
@@ -1437,6 +1775,19 @@ public abstract class FSEditLogOp {
       builder.append("]");
       return builder.toString();
     }
+    
+    @Override
+    protected void toXml(ContentHandler contentHandler) throws SAXException {
+      XMLUtils.addSaxString(contentHandler, "LEASEHOLDER", leaseHolder);
+      XMLUtils.addSaxString(contentHandler, "PATH", path);
+      XMLUtils.addSaxString(contentHandler, "NEWHOLDER", newHolder);
+    }
+    
+    @Override void fromXml(Stanza st) throws InvalidXmlException {
+      this.leaseHolder = st.getValue("LEASEHOLDER");
+      this.path = st.getValue("PATH");
+      this.newHolder = st.getValue("NEWHOLDER");
+    }
   }
 
   static class GetDelegationTokenOp extends FSEditLogOp {
@@ -1463,7 +1814,8 @@ public abstract class FSEditLogOp {
       return this;
     }
 
-    @Override 
+    @Override
+    public 
     void writeFields(DataOutputStream out) throws IOException {
       token.write(out);
       FSImageSerialization.writeLong(expiryTime, out);
@@ -1495,6 +1847,19 @@ public abstract class FSEditLogOp {
       builder.append("]");
       return builder.toString();
     }
+    
+    @Override
+    protected void toXml(ContentHandler contentHandler) throws SAXException {
+      FSEditLogOp.delegationTokenToXml(contentHandler, token);
+      XMLUtils.addSaxString(contentHandler, "EXPIRY_TIME",
+          Long.valueOf(expiryTime).toString());
+    }
+    
+    @Override void fromXml(Stanza st) throws InvalidXmlException {
+      this.token = delegationTokenFromXml(st.getChildren(
+          "DELEGATION_TOKEN_IDENTIFIER").get(0));
+      this.expiryTime = Long.valueOf(st.getValue("EXPIRY_TIME"));
+    }
   }
 
   static class RenewDelegationTokenOp extends FSEditLogOp {
@@ -1521,7 +1886,8 @@ public abstract class FSEditLogOp {
       return this;
     }
 
-    @Override 
+    @Override
+    public 
     void writeFields(DataOutputStream out) throws IOException {
       token.write(out);
       FSImageSerialization.writeLong(expiryTime, out);
@@ -1553,6 +1919,19 @@ public abstract class FSEditLogOp {
       builder.append("]");
       return builder.toString();
     }
+    
+    @Override
+    protected void toXml(ContentHandler contentHandler) throws SAXException {
+      FSEditLogOp.delegationTokenToXml(contentHandler, token);
+      XMLUtils.addSaxString(contentHandler, "EXPIRY_TIME",
+          Long.valueOf(expiryTime).toString());
+    }
+    
+    @Override void fromXml(Stanza st) throws InvalidXmlException {
+      this.token = delegationTokenFromXml(st.getChildren(
+          "DELEGATION_TOKEN_IDENTIFIER").get(0));
+      this.expiryTime = Long.valueOf(st.getValue("EXPIRY_TIME"));
+    }
   }
 
   static class CancelDelegationTokenOp extends FSEditLogOp {
@@ -1573,7 +1952,8 @@ public abstract class FSEditLogOp {
       return this;
     }
 
-    @Override 
+    @Override
+    public 
     void writeFields(DataOutputStream out) throws IOException {
       token.write(out);
     }
@@ -1597,6 +1977,16 @@ public abstract class FSEditLogOp {
       builder.append("]");
       return builder.toString();
     }
+    
+    @Override
+    protected void toXml(ContentHandler contentHandler) throws SAXException {
+      FSEditLogOp.delegationTokenToXml(contentHandler, token);
+    }
+    
+    @Override void fromXml(Stanza st) throws InvalidXmlException {
+      this.token = delegationTokenFromXml(st.getChildren(
+          "DELEGATION_TOKEN_IDENTIFIER").get(0));
+    }
   }
 
   static class UpdateMasterKeyOp extends FSEditLogOp {
@@ -1616,7 +2006,8 @@ public abstract class FSEditLogOp {
       return this;
     }
     
-    @Override 
+    @Override
+    public 
     void writeFields(DataOutputStream out) throws IOException {
       key.write(out);
     }
@@ -1640,6 +2031,16 @@ public abstract class FSEditLogOp {
       builder.append("]");
       return builder.toString();
     }
+    
+    @Override
+    protected void toXml(ContentHandler contentHandler) throws SAXException {
+      FSEditLogOp.delegationKeyToXml(contentHandler, key);
+    }
+    
+    @Override void fromXml(Stanza st) throws InvalidXmlException {
+      this.key = delegationKeyFromXml(st.getChildren(
+          "DELEGATION_KEY").get(0));
+    }
   }
   
   static class LogSegmentOp extends FSEditLogOp {
@@ -1659,6 +2060,7 @@ public abstract class FSEditLogOp {
     }
 
     @Override
+    public
     void writeFields(DataOutputStream out) throws IOException {
       // no data stored
     }
@@ -1673,6 +2075,15 @@ public abstract class FSEditLogOp {
       builder.append("]");
       return builder.toString();
     }
+
+    @Override
+    protected void toXml(ContentHandler contentHandler) throws SAXException {
+      // no data stored
+    }
+    
+    @Override void fromXml(Stanza st) throws InvalidXmlException {
+      // do nothing
+    }
   }
 
   static class InvalidOp extends FSEditLogOp {
@@ -1684,7 +2095,8 @@ public abstract class FSEditLogOp {
       return (InvalidOp)opInstances.get().get(OP_INVALID);
     }
 
-    @Override 
+    @Override
+    public 
     void writeFields(DataOutputStream out) throws IOException {
     }
     
@@ -1704,6 +2116,14 @@ public abstract class FSEditLogOp {
       builder.append("]");
       return builder.toString();
     }
+    @Override
+    protected void toXml(ContentHandler contentHandler) throws SAXException {
+      // no data stored
+    }
+    
+    @Override void fromXml(Stanza st) throws InvalidXmlException {
+      // do nothing
+    }
   }
 
   static private short readShort(DataInputStream in) throws IOException {
@@ -1874,4 +2294,140 @@ public abstract class FSEditLogOp {
       }
     }
   }
+
+  public void outputToXml(ContentHandler contentHandler) throws SAXException {
+    contentHandler.startElement("", "", "RECORD", new AttributesImpl());
+    XMLUtils.addSaxString(contentHandler, "OPCODE", opCode.toString());
+    contentHandler.startElement("", "", "DATA", new AttributesImpl());
+    XMLUtils.addSaxString(contentHandler, "TXID", "" + txid);
+    toXml(contentHandler);
+    contentHandler.endElement("", "", "DATA");
+    contentHandler.endElement("", "", "RECORD");
+  }
+
+  protected abstract void toXml(ContentHandler contentHandler)
+      throws SAXException;
+  
+  abstract void fromXml(Stanza st) throws InvalidXmlException;
+  
+  public void decodeXml(Stanza st) throws InvalidXmlException {
+    this.txid = Long.valueOf(st.getValue("TXID"));
+    fromXml(st);
+  }
+  
+  public static void blockToXml(ContentHandler contentHandler, Block block) 
+      throws SAXException {
+    contentHandler.startElement("", "", "BLOCK", new AttributesImpl());
+    XMLUtils.addSaxString(contentHandler, "BLOCK_ID",
+        Long.valueOf(block.getBlockId()).toString());
+    XMLUtils.addSaxString(contentHandler, "NUM_BYTES",
+        Long.valueOf(block.getNumBytes()).toString());
+    XMLUtils.addSaxString(contentHandler, "GENERATION_STAMP",
+        Long.valueOf(block.getGenerationStamp()).toString());
+    contentHandler.endElement("", "", "BLOCK");
+  }
+
+  public static Block blockFromXml(Stanza st)
+      throws InvalidXmlException {
+    long blockId = Long.valueOf(st.getValue("BLOCK_ID"));
+    long numBytes = Long.valueOf(st.getValue("NUM_BYTES"));
+    long generationStamp = Long.valueOf(st.getValue("GENERATION_STAMP"));
+    return new Block(blockId, numBytes, generationStamp);
+  }
+
+  public static void delegationTokenToXml(ContentHandler contentHandler,
+      DelegationTokenIdentifier token) throws SAXException {
+    contentHandler.startElement("", "", "DELEGATION_TOKEN_IDENTIFIER", new AttributesImpl());
+    XMLUtils.addSaxString(contentHandler, "KIND", token.getKind().toString());
+    XMLUtils.addSaxString(contentHandler, "SEQUENCE_NUMBER",
+        Integer.valueOf(token.getSequenceNumber()).toString());
+    XMLUtils.addSaxString(contentHandler, "OWNER",
+        token.getOwner().toString());
+    XMLUtils.addSaxString(contentHandler, "RENEWER",
+        token.getRenewer().toString());
+    XMLUtils.addSaxString(contentHandler, "REALUSER",
+        token.getRealUser().toString());
+    XMLUtils.addSaxString(contentHandler, "ISSUE_DATE",
+        Long.valueOf(token.getIssueDate()).toString());
+    XMLUtils.addSaxString(contentHandler, "MAX_DATE",
+        Long.valueOf(token.getMaxDate()).toString());
+    XMLUtils.addSaxString(contentHandler, "MASTER_KEY_ID",
+        Integer.valueOf(token.getMasterKeyId()).toString());
+    contentHandler.endElement("", "", "DELEGATION_TOKEN_IDENTIFIER");
+  }
+
+  public static DelegationTokenIdentifier delegationTokenFromXml(Stanza st)
+      throws InvalidXmlException {
+    String kind = st.getValue("KIND");
+    if (!kind.equals(DelegationTokenIdentifier.
+        HDFS_DELEGATION_KIND.toString())) {
+      throw new InvalidXmlException("can't understand " +
+        "DelegationTokenIdentifier KIND " + kind);
+    }
+    int seqNum = Integer.valueOf(st.getValue("SEQUENCE_NUMBER"));
+    String owner = st.getValue("OWNER");
+    String renewer = st.getValue("RENEWER");
+    String realuser = st.getValue("REALUSER");
+    long issueDate = Long.valueOf(st.getValue("ISSUE_DATE"));
+    long maxDate = Long.valueOf(st.getValue("MAX_DATE"));
+    int masterKeyId = Integer.valueOf(st.getValue("MASTER_KEY_ID"));
+    DelegationTokenIdentifier token =
+        new DelegationTokenIdentifier(new Text(owner),
+            new Text(renewer), new Text(realuser));
+    token.setSequenceNumber(seqNum);
+    token.setIssueDate(issueDate);
+    token.setMaxDate(maxDate);
+    token.setMasterKeyId(masterKeyId);
+    return token;
+  }
+
+  public static void delegationKeyToXml(ContentHandler contentHandler,
+      DelegationKey key) throws SAXException {
+    contentHandler.startElement("", "", "DELEGATION_KEY", new AttributesImpl());
+    XMLUtils.addSaxString(contentHandler, "KEY_ID",
+        Integer.valueOf(key.getKeyId()).toString());
+    XMLUtils.addSaxString(contentHandler, "EXPIRY_DATE",
+        Long.valueOf(key.getExpiryDate()).toString());
+    if (key.getEncodedKey() != null) {
+      XMLUtils.addSaxString(contentHandler, "KEY",
+          Hex.encodeHexString(key.getEncodedKey()));
+    }
+    contentHandler.endElement("", "", "DELEGATION_KEY");
+  }
+  
+  public static DelegationKey delegationKeyFromXml(Stanza st)
+      throws InvalidXmlException {
+    int keyId = Integer.valueOf(st.getValue("KEY_ID"));
+    long expiryDate = Long.valueOf(st.getValue("EXPIRY_DATE"));
+    byte key[] = null;
+    try {
+      key = Hex.decodeHex(st.getValue("KEY").toCharArray());
+    } catch (DecoderException e) {
+      throw new InvalidXmlException(e.toString());
+    } catch (InvalidXmlException e) {
+    }
+    return new DelegationKey(keyId, expiryDate, key);
+  }
+
+  public static void permissionStatusToXml(ContentHandler contentHandler,
+      PermissionStatus perm) throws SAXException {
+    contentHandler.startElement("", "", "PERMISSION_STATUS", new AttributesImpl());
+    XMLUtils.addSaxString(contentHandler, "USERNAME", perm.getUserName());
+    XMLUtils.addSaxString(contentHandler, "GROUPNAME", perm.getGroupName());
+    XMLUtils.addSaxString(contentHandler, "MODE",
+        Short.valueOf(perm.getPermission().toShort()).toString());
+    contentHandler.endElement("", "", "PERMISSION_STATUS");
+  }
+
+  public static PermissionStatus permissionStatusFromXml(Stanza st)
+      throws InvalidXmlException {
+    String username = st.getValue("USERNAME");
+    String groupname = st.getValue("GROUPNAME");
+    short mode = Short.valueOf(st.getValue("MODE"));
+    return new PermissionStatus(username, groupname, new FsPermission(mode));
+  }
+
+  public static FSEditLogOp getOpInstance(FSEditLogOpCodes opCode) {
+    return opInstances.get().get(opCode);
+  }
 }

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

@@ -164,6 +164,7 @@ import org.apache.hadoop.hdfs.server.namenode.ha.HAState;
 import org.apache.hadoop.hdfs.server.namenode.ha.StandbyCheckpointer;
 import org.apache.hadoop.hdfs.server.namenode.ha.StandbyState;
 import org.apache.hadoop.hdfs.server.namenode.metrics.FSNamesystemMBean;
+import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.HeartbeatResponse;
@@ -346,10 +347,27 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    * @throws IOException if loading fails
    */
   public static FSNamesystem loadFromDisk(Configuration conf)
-    throws IOException {
+      throws IOException {
     Collection<URI> namespaceDirs = FSNamesystem.getNamespaceDirs(conf);
     List<URI> namespaceEditsDirs = 
       FSNamesystem.getNamespaceEditsDirs(conf);
+    return loadFromDisk(conf, namespaceDirs, namespaceEditsDirs);
+  }
+
+  /**
+   * Instantiates an FSNamesystem loaded from the image and edits
+   * directories passed.
+   * 
+   * @param conf the Configuration which specifies the storage directories
+   *             from which to load
+   * @param namespaceDirs directories to load the fsimages
+   * @param namespaceEditsDirs directories to load the edits from
+   * @return an FSNamesystem which contains the loaded namespace
+   * @throws IOException if loading fails
+   */
+  public static FSNamesystem loadFromDisk(Configuration conf,
+      Collection<URI> namespaceDirs, List<URI> namespaceEditsDirs)
+      throws IOException {
 
     if (namespaceDirs.size() == 1) {
       LOG.warn("Only one " + DFS_NAMENODE_NAME_DIR_KEY
@@ -370,8 +388,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       HAUtil.isHAEnabled(conf, nameserviceId));
     long timeTakenToLoadFSImage = now() - loadStart;
     LOG.info("Finished loading FSImage in " + timeTakenToLoadFSImage + " msecs");
-    NameNode.getNameNodeMetrics().setFsImageLoadTime(
-                              (int) timeTakenToLoadFSImage);
+    NameNodeMetrics nnMetrics = NameNode.getNameNodeMetrics();
+    if (nnMetrics != null) {
+      nnMetrics.setFsImageLoadTime((int) timeTakenToLoadFSImage);
+    }
     return namesystem;
   }
 

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

@@ -62,7 +62,7 @@ public class FileChecksumServlets {
           ? ((DatanodeInfo)host).getHostName() : host.getIpAddr();
       final String scheme = request.getScheme();
       final int port = "https".equals(scheme)
-          ? (Integer)getServletContext().getAttribute("datanode.https.port")
+          ? (Integer)getServletContext().getAttribute(DFSConfigKeys.DFS_DATANODE_HTTPS_PORT_KEY)
           : host.getInfoPort();
       final String encodedPath = ServletUtil.getRawPath(request, "/fileChecksum");
 

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

@@ -27,6 +27,7 @@ import javax.servlet.http.HttpServletResponse;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -62,7 +63,7 @@ public class FileDataServlet extends DfsServlet {
       hostname = host.getIpAddr();
     }
     final int port = "https".equals(scheme)
-      ? (Integer)getServletContext().getAttribute("datanode.https.port")
+      ? (Integer)getServletContext().getAttribute(DFSConfigKeys.DFS_DATANODE_HTTPS_PORT_KEY)
       : host.getInfoPort();
 
     String dtParam = "";

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

@@ -40,6 +40,8 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Trash;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
+
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
@@ -58,6 +60,7 @@ import org.apache.hadoop.hdfs.server.protocol.JournalProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.ipc.StandbyException;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.net.NetUtils;
@@ -70,6 +73,9 @@ import org.apache.hadoop.tools.GetUserMappingsProtocol;
 import org.apache.hadoop.util.ServicePlugin;
 import org.apache.hadoop.util.StringUtils;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
+
 /**********************************************************
  * NameNode serves as both directory namespace manager and
  * "inode table" for the Hadoop DFS.  There is a single NameNode
@@ -156,7 +162,8 @@ public class NameNode {
     DFS_SECONDARY_NAMENODE_KEYTAB_FILE_KEY,
     DFS_NAMENODE_BACKUP_ADDRESS_KEY,
     DFS_NAMENODE_BACKUP_HTTP_ADDRESS_KEY,
-    DFS_NAMENODE_BACKUP_SERVICE_RPC_ADDRESS_KEY
+    DFS_NAMENODE_BACKUP_SERVICE_RPC_ADDRESS_KEY,
+    DFS_HA_FENCE_METHODS_KEY
   };
   
   public long getProtocolVersion(String protocol, 
@@ -729,6 +736,67 @@ public class NameNode {
                 + "to true in order to format this filesystem");
     }
   }
+  
+  @VisibleForTesting
+  public static boolean initializeSharedEdits(Configuration conf) {
+    return initializeSharedEdits(conf, true);
+  }
+  
+  @VisibleForTesting
+  public static boolean initializeSharedEdits(Configuration conf,
+      boolean force) {
+    return initializeSharedEdits(conf, force, false);
+  }
+  
+  /**
+   * Format a new shared edits dir.
+   * 
+   * @param conf configuration
+   * @param force format regardless of whether or not the shared edits dir exists
+   * @param interactive prompt the user when a dir exists
+   * @return true if the command aborts, false otherwise
+   */
+  private static boolean initializeSharedEdits(Configuration conf,
+      boolean force, boolean interactive) {
+    NNStorage existingStorage = null;
+    try {
+      FSNamesystem fsns = FSNamesystem.loadFromDisk(conf,
+          FSNamesystem.getNamespaceDirs(conf),
+          FSNamesystem.getNamespaceEditsDirs(conf, false));
+      
+      existingStorage = fsns.getFSImage().getStorage();
+      
+      Collection<URI> sharedEditsDirs = FSNamesystem.getSharedEditsDirs(conf);
+      if (!confirmFormat(sharedEditsDirs, force, interactive)) {
+        return true; // aborted
+      }
+      NNStorage newSharedStorage = new NNStorage(conf,
+          Lists.<URI>newArrayList(),
+          sharedEditsDirs);
+      
+      newSharedStorage.format(new NamespaceInfo(
+          existingStorage.getNamespaceID(),
+          existingStorage.getClusterID(),
+          existingStorage.getBlockPoolID(),
+          existingStorage.getCTime(),
+          existingStorage.getDistributedUpgradeVersion()));
+    } catch (Exception e) {
+      LOG.error("Could not format shared edits dir", e);
+      return true; // aborted
+    } finally {
+      // Have to unlock storage explicitly for the case when we're running in a
+      // unit test, which runs in the same JVM as NNs.
+      if (existingStorage != null) {
+        try {
+          existingStorage.unlockAll();
+        } catch (IOException ioe) {
+          LOG.warn("Could not unlock storage directories", ioe);
+          return true; // aborted
+        }
+      }
+    }
+    return false; // did not abort
+  }
 
   private static boolean finalize(Configuration conf,
                                boolean isConfirmationNeeded
@@ -763,7 +831,8 @@ public class NameNode {
       StartupOption.ROLLBACK.getName() + "] | [" +
       StartupOption.FINALIZE.getName() + "] | [" +
       StartupOption.IMPORT.getName() + "] | [" +
-      StartupOption.BOOTSTRAPSTANDBY.getName() + "]");
+      StartupOption.BOOTSTRAPSTANDBY.getName() + "] | [" +
+      StartupOption.INITIALIZESHAREDEDITS.getName() + "]");
   }
 
   private static StartupOption parseArguments(String args[]) {
@@ -804,6 +873,9 @@ public class NameNode {
       } else if (StartupOption.BOOTSTRAPSTANDBY.getName().equalsIgnoreCase(cmd)) {
         startOpt = StartupOption.BOOTSTRAPSTANDBY;
         return startOpt;
+      } else if (StartupOption.INITIALIZESHAREDEDITS.getName().equalsIgnoreCase(cmd)) {
+        startOpt = StartupOption.INITIALIZESHAREDEDITS;
+        return startOpt;
       } else {
         return null;
       }
@@ -868,29 +940,39 @@ public class NameNode {
     }
 
     switch (startOpt) {
-      case FORMAT:
+      case FORMAT: {
         boolean aborted = format(conf, false);
         System.exit(aborted ? 1 : 0);
         return null; // avoid javac warning
-      case GENCLUSTERID:
+      }
+      case GENCLUSTERID: {
         System.err.println("Generating new cluster id:");
         System.out.println(NNStorage.newClusterID());
         System.exit(0);
         return null;
-      case FINALIZE:
-        aborted = finalize(conf, true);
+      }
+      case FINALIZE: {
+        boolean aborted = finalize(conf, true);
         System.exit(aborted ? 1 : 0);
         return null; // avoid javac warning
-      case BOOTSTRAPSTANDBY:
+      }
+      case BOOTSTRAPSTANDBY: {
         String toolArgs[] = Arrays.copyOfRange(argv, 1, argv.length);
         int rc = BootstrapStandby.run(toolArgs, conf);
         System.exit(rc);
         return null; // avoid warning
+      }
+      case INITIALIZESHAREDEDITS: {
+        boolean aborted = initializeSharedEdits(conf, false, true);
+        System.exit(aborted ? 1 : 0);
+        return null; // avoid warning
+      }
       case BACKUP:
-      case CHECKPOINT:
+      case CHECKPOINT: {
         NamenodeRole role = startOpt.toNodeRole();
         DefaultMetricsSystem.initialize(role.toString().replace(" ", ""));
         return new BackupNode(conf, role);
+      }
       default:
         DefaultMetricsSystem.initialize("NameNode");
         return new NameNode(conf);

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

@@ -165,10 +165,11 @@ public class NameNodeHttpServer {
             httpServer.addSslListener(secInfoSocAddr, sslConf, needClientAuth,
                 useKrb);
             // assume same ssl port for all datanodes
-            InetSocketAddress datanodeSslPort = NetUtils.createSocketAddr(conf
-                .get(DFS_DATANODE_HTTPS_ADDRESS_KEY, infoHost + ":" + 50475));
-            httpServer.setAttribute("datanode.https.port", datanodeSslPort
-                .getPort());
+            InetSocketAddress datanodeSslPort = NetUtils.createSocketAddr(
+                conf.get(DFS_DATANODE_HTTPS_ADDRESS_KEY,
+                    infoHost + ":" + DFSConfigKeys.DFS_DATANODE_HTTPS_DEFAULT_PORT));
+            httpServer.setAttribute(DFSConfigKeys.DFS_DATANODE_HTTPS_PORT_KEY,
+                datanodeSslPort.getPort());
           }
           httpServer.setAttribute(NAMENODE_ATTRIBUTE_KEY, nn);
           httpServer.setAttribute(NAMENODE_ADDRESS_ATTRIBUTE_KEY,

+ 11 - 15
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeRegistration.java

@@ -24,6 +24,7 @@ import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.server.common.Storage;
@@ -52,18 +53,9 @@ implements Writable, NodeRegistration {
   private StorageInfo storageInfo;
   private ExportedBlockKeys exportedKeys;
 
-  /**
-   * Default constructor.
-   */
   public DatanodeRegistration() {
-    this("");
-  }
-  
-  /**
-   * Create DatanodeRegistration
-   */
-  public DatanodeRegistration(String ipAddr) {
-    this(ipAddr, new StorageInfo(), new ExportedBlockKeys());
+    this("", DFSConfigKeys.DFS_DATANODE_HTTP_DEFAULT_PORT,
+        new StorageInfo(), new ExportedBlockKeys());
   }
   
   public DatanodeRegistration(DatanodeID dn, StorageInfo info,
@@ -72,10 +64,14 @@ implements Writable, NodeRegistration {
     this.storageInfo = info;
     this.exportedKeys = keys;
   }
-  
-  public DatanodeRegistration(String ipAddr, StorageInfo info,
+
+  public DatanodeRegistration(String ipAddr, int xferPort) {
+    this(ipAddr, xferPort, new StorageInfo(), new ExportedBlockKeys());
+  }
+
+  public DatanodeRegistration(String ipAddr, int xferPort, StorageInfo info,
       ExportedBlockKeys keys) {
-    super(ipAddr);
+    super(ipAddr, xferPort);
     this.storageInfo = info;
     this.exportedKeys = keys;
   }
@@ -114,7 +110,7 @@ implements Writable, NodeRegistration {
   @Override
   public String toString() {
     return getClass().getSimpleName()
-      + "(" + ipAddr
+      + "(" + getIpAddr()
       + ", storageID=" + storageID
       + ", infoPort=" + infoPort
       + ", ipcPort=" + ipcPort

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

@@ -50,14 +50,19 @@ public class NamespaceInfo extends StorageInfo {
     super();
     buildVersion = null;
   }
-  
-  public NamespaceInfo(int nsID, String clusterID, String bpID, 
-      long cT, int duVersion) {
+
+  public NamespaceInfo(int nsID, String clusterID, String bpID,
+      long cT, int duVersion, String buildVersion) {
     super(HdfsConstants.LAYOUT_VERSION, nsID, clusterID, cT);
     blockPoolID = bpID;
-    buildVersion = Storage.getBuildVersion();
+    this.buildVersion = buildVersion;
     this.distributedUpgradeVersion = duVersion;
   }
+
+  public NamespaceInfo(int nsID, String clusterID, String bpID, 
+      long cT, int duVersion) {
+    this(nsID, clusterID, bpID, cT, duVersion, Storage.getBuildVersion());
+  }
   
   public String getBuildVersion() {
     return buildVersion;

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/NNHAServiceTarget.java

@@ -25,6 +25,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.ha.BadFencingConfigurationException;
 import org.apache.hadoop.ha.HAServiceTarget;
 import org.apache.hadoop.ha.NodeFencer;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
@@ -76,7 +77,8 @@ public class NNHAServiceTarget extends HAServiceTarget {
     this.addr = NetUtils.createSocketAddr(serviceAddr,
         NameNode.DEFAULT_PORT);
     try {
-      this.fencer = NodeFencer.create(targetConf);
+      this.fencer = NodeFencer.create(targetConf,
+          DFSConfigKeys.DFS_HA_FENCE_METHODS_KEY);
     } catch (BadFencingConfigurationException e) {
       this.fenceConfigError = e;
     }

+ 17 - 70
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/BinaryEditsVisitor.java

@@ -17,104 +17,51 @@
  */
 package org.apache.hadoop.hdfs.tools.offlineEditsViewer;
 
-import java.io.FileOutputStream;
-import java.io.DataOutputStream;
+import java.io.File;
 import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
+import org.apache.hadoop.hdfs.server.namenode.EditLogFileOutputStream;
 
 /**
  * BinaryEditsVisitor implements a binary EditsVisitor
  */
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
-public class BinaryEditsVisitor extends EditsVisitor {
-  final private DataOutputStream out;
+public class BinaryEditsVisitor implements OfflineEditsVisitor {
+  final private EditLogFileOutputStream elfos;
 
   /**
-   * Create a processor that writes to a given file and
-   * reads using a given Tokenizer
+   * Create a processor that writes to a given file
    *
    * @param filename Name of file to write output to
-   * @param tokenizer Input tokenizer
    */
-  public BinaryEditsVisitor(String filename, Tokenizer tokenizer)
-    throws IOException {
-
-    this(filename, tokenizer, false);
-  }
-
-  /**
-   * Create a processor that writes to a given file and reads using
-   * a given Tokenizer, may also print to screen
-   *
-   * @param filename Name of file to write output to
-   * @param tokenizer Input tokenizer
-   * @param printToScreen Mirror output to screen? (ignored for binary)
-   */
-  public BinaryEditsVisitor(String filename,
-    Tokenizer tokenizer,
-    boolean printToScreen) throws IOException {
-
-    super(tokenizer);
-    out = new DataOutputStream(new FileOutputStream(filename));
+  public BinaryEditsVisitor(String outputName) throws IOException {
+    this.elfos = new EditLogFileOutputStream(new File(outputName), 0);
+    elfos.create();
   }
 
   /**
    * Start the visitor (initialization)
    */
   @Override
-  void start() throws IOException {
-    // nothing to do for binary format
+  public void start(int version) throws IOException {
   }
 
   /**
    * Finish the visitor
    */
   @Override
-  void finish() throws IOException {
-    close();
-  }
-
-  /**
-   * Finish the visitor and indicate an error
-   */
-  @Override
-  void finishAbnormally() throws IOException {
-    System.err.println("Error processing EditLog file.  Exiting.");
-    close();
-  }
-
-  /**
-   * Close output stream and prevent further writing
-   */
-  private void close() throws IOException {
-    out.close();
-  }
-
-  /**
-   * Visit a enclosing element (element that has other elements in it)
-   */
-  @Override
-  void visitEnclosingElement(Tokenizer.Token value) throws IOException {
-    // nothing to do for binary format
+  public void close(Throwable error) throws IOException {
+    elfos.setReadyToFlush();
+    elfos.flushAndSync();
+    elfos.close();
   }
 
-  /**
-   * End of eclosing element
-   */
-  @Override
-  void leaveEnclosingElement() throws IOException {
-    // nothing to do for binary format
-  }  
-
-  /**
-   * Visit a Token
-   */
   @Override
-  Tokenizer.Token visit(Tokenizer.Token value) throws IOException {
-    value.toBinary(out);
-    return value;
+  public void visitOp(FSEditLogOp op) throws IOException {
+    elfos.write(op);
   }
-}
+}

+ 0 - 88
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsElement.java

@@ -1,88 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.tools.offlineEditsViewer;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * Structural elements of an EditLog that may be encountered within the
- * file. EditsVisitor is able to process these elements.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public enum EditsElement {
-  EDITS,
-  EDITS_VERSION,
-  RECORD,
-  OPCODE,
-  TRANSACTION_ID,
-  DATA,
-    // elements in the data part of the editLog records
-    LENGTH,
-    // for OP_SET_GENSTAMP
-    GENERATION_STAMP,
-    // for OP_ADD, OP_CLOSE
-    PATH,
-    REPLICATION,
-    MTIME,
-    ATIME,
-    BLOCKSIZE,
-    NUMBLOCKS,
-    BLOCK,
-      BLOCK_ID,
-      BLOCK_NUM_BYTES,
-      BLOCK_GENERATION_STAMP,
-      BLOCK_DELTA_NUM_BYTES, // delta-encoded relative to previous block
-      BLOCK_DELTA_GEN_STAMP, // delta-encoded relative to previous block
-    PERMISSION_STATUS,
-      FS_PERMISSIONS,
-    CLIENT_NAME,
-    CLIENT_MACHINE,
-    // for OP_RENAME_OLD
-    SOURCE,
-    DESTINATION,
-    TIMESTAMP,
-    // for OP_SET_OWNER
-    USERNAME,
-    GROUPNAME,
-    // for OP_SET_QUOTA
-    NS_QUOTA,
-    DS_QUOTA,
-    // for OP_RENAME
-    RENAME_OPTIONS,
-    // for OP_CONCAT_DELETE
-    CONCAT_TARGET,
-    CONCAT_SOURCE,
-    // for OP_GET_DELEGATION_TOKEN
-    T_VERSION,
-    T_OWNER,
-    T_RENEWER,
-    T_REAL_USER,
-    T_ISSUE_DATE,
-    T_MAX_DATE,
-    T_SEQUENCE_NUMBER,
-    T_MASTER_KEY_ID,
-    T_EXPIRY_TIME,
-    // for OP_UPDATE_MASTER_KEY
-    KEY_ID,
-    KEY_EXPIRY_DATE,
-    KEY_LENGTH,
-    KEY_BLOB,
-    CHECKSUM
-}

+ 0 - 74
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsLoader.java

@@ -1,74 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.tools.offlineEditsViewer;
-
-import java.io.IOException;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * An EditsLoader can read a Hadoop EditLog file  and walk over its
- * structure using the supplied EditsVisitor.
- *
- * Each implementation of EditsLoader is designed to rapidly process an
- * edits log file.  As long as minor changes are made from one layout version
- * to another, it is acceptable to tweak one implementation to read the next.
- * However, if the layout version changes enough that it would make a
- * processor slow or difficult to read, another processor should be created.
- * This allows each processor to quickly read an edits log without getting
- * bogged down in dealing with significant differences between layout versions.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-interface EditsLoader {
-
-  /**
-   * Loads the edits file
-   */
-  public void loadEdits() throws IOException;
-
-  /**
-   * Can this processor handle the specified version of EditLog file?
-   *
-   * @param version EditLog version file
-   * @return True if this instance can process the file
-   */
-  public boolean canLoadVersion(int version);
-
-  /**
-   * Factory for obtaining version of edits log loader that can read
-   * a particular edits log format.
-   */
-  @InterfaceAudience.Private
-  @InterfaceStability.Unstable
-  public class LoaderFactory {
-    // Java doesn't support static methods on interfaces, which necessitates
-    // this factory class
-
-    /**
-     * Create an edits log loader, at this point we only have one,
-     * we might need to add more later
-     *
-     * @param v an instance of EditsVisitor (binary, XML etc.)
-     * @return EditsLoader that can interpret specified version, or null
-     */
-    static public EditsLoader getLoader(EditsVisitor v) {
-      return new EditsLoaderCurrent(v);
-    }
-  }
-}

+ 0 - 608
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsLoaderCurrent.java

@@ -1,608 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.tools.offlineEditsViewer;
-
-import java.io.EOFException;
-import java.io.IOException;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-import org.apache.hadoop.hdfs.protocol.LayoutVersion;
-import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes;
-
-import static org.apache.hadoop.hdfs.tools.offlineEditsViewer.Tokenizer.ByteToken;
-import static org.apache.hadoop.hdfs.tools.offlineEditsViewer.Tokenizer.IntToken;
-import static org.apache.hadoop.hdfs.tools.offlineEditsViewer.Tokenizer.VIntToken;
-
-/**
- * EditsLoaderCurrent processes Hadoop EditLogs files and walks over
- * them using a provided EditsVisitor, calling the visitor at each element
- * enumerated below.
- *
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-class EditsLoaderCurrent implements EditsLoader {
-
-  private static int[] supportedVersions = { -18, -19, -20, -21, -22, -23, -24,
-      -25, -26, -27, -28, -30, -31, -32, -33, -34, -35, -36, -37, -38, -39, -40};
-
-  private EditsVisitor v;
-  private int editsVersion = 0;
-
-  /**
-   * Constructor
-   */
-  public EditsLoaderCurrent(EditsVisitor visitor) {
-    v = visitor;
-  }
-
-  /**
-   * Checks if this EditsLoader can load given version of edits
-   *
-   * @param version version to load
-   * @return true if this EditsLoader can load given version of edits
-   */
-  @Override
-  public boolean canLoadVersion(int version) {
-    for(int v : supportedVersions) { if(v == version) return true; }
-    return false;
-  }
-
-  /**
-   * Visit a transaction ID, if the log version supports it.
-   */
-  private void visitTxId() throws IOException {
-    if (LayoutVersion.supports(Feature.STORED_TXIDS, editsVersion)) {
-      v.visitLong(EditsElement.TRANSACTION_ID);
-    }
-  }
-  
-  /**
-   * Visit OP_INVALID
-   */
-  private void visit_OP_INVALID() throws IOException {
-  }
-
-  /**
-   * Visit OP_ADD
-   */
-  private void visit_OP_ADD() throws IOException {
-    visit_OP_ADD_or_OP_CLOSE(FSEditLogOpCodes.OP_ADD);
-  }
-
-  /**
-   * Visit OP_CLOSE
-   */
-  private void visit_OP_CLOSE() throws IOException {
-    visit_OP_ADD_or_OP_CLOSE(FSEditLogOpCodes.OP_CLOSE);
-  }
-
-  /**
-   * Visit OP_ADD and OP_CLOSE, they are almost the same
-   *
-   * @param editsOpCode op code to visit
-   */
-  private void visit_OP_ADD_or_OP_CLOSE(FSEditLogOpCodes editsOpCode)
-    throws IOException {
-    visitTxId();
-    if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
-      IntToken opAddLength = v.visitInt(EditsElement.LENGTH);
-      // this happens if the edits is not properly ended (-1 op code),
-      // it is padded at the end with all zeros, OP_ADD is zero so
-      // without this check we would treat all zeros as empty OP_ADD)
-      if (opAddLength.value == 0) {
-        throw new IOException("OpCode " + editsOpCode
-            + " has zero length (corrupted edits)");
-      }
-    }
-    
-    v.visitStringUTF8(EditsElement.PATH);
-    if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
-      v.visitShort(EditsElement.REPLICATION);
-      v.visitLong(EditsElement.MTIME);
-      v.visitLong(EditsElement.ATIME);
-      v.visitLong(EditsElement.BLOCKSIZE);
-    } else {
-      v.visitStringUTF8(EditsElement.REPLICATION);
-      v.visitStringUTF8(EditsElement.MTIME);
-      v.visitStringUTF8(EditsElement.ATIME);
-      v.visitStringUTF8(EditsElement.BLOCKSIZE);
-    }
-    // now read blocks
-    IntToken numBlocksToken = v.visitInt(EditsElement.NUMBLOCKS);
-    for (int i = 0; i < numBlocksToken.value; i++) {
-      v.visitEnclosingElement(EditsElement.BLOCK);
-
-      v.visitLong(EditsElement.BLOCK_ID);
-      v.visitLong(EditsElement.BLOCK_NUM_BYTES);
-      v.visitLong(EditsElement.BLOCK_GENERATION_STAMP);
-
-      v.leaveEnclosingElement();
-    }
-    // PERMISSION_STATUS
-    v.visitEnclosingElement(EditsElement.PERMISSION_STATUS);
-
-    v.visitStringText( EditsElement.USERNAME);
-    v.visitStringText( EditsElement.GROUPNAME);
-    v.visitShort(      EditsElement.FS_PERMISSIONS);
-
-    v.leaveEnclosingElement();
-    if(editsOpCode == FSEditLogOpCodes.OP_ADD) {
-      v.visitStringUTF8(EditsElement.CLIENT_NAME);
-      v.visitStringUTF8(EditsElement.CLIENT_MACHINE);
-    }
-  }
-
-  private void visit_OP_UPDATE_BLOCKS() throws IOException {
-    visitTxId();
-    v.visitStringUTF8(EditsElement.PATH);
-    VIntToken numBlocksToken = v.visitVInt(EditsElement.NUMBLOCKS);
-    for (int i = 0; i < numBlocksToken.value; i++) {
-      v.visitEnclosingElement(EditsElement.BLOCK);
-
-      v.visitLong(EditsElement.BLOCK_ID);
-      if (i == 0) {
-        v.visitVLong(EditsElement.BLOCK_NUM_BYTES);
-        v.visitVLong(EditsElement.BLOCK_GENERATION_STAMP);
-      } else {
-        v.visitVLong(EditsElement.BLOCK_DELTA_NUM_BYTES);
-        v.visitVLong(EditsElement.BLOCK_DELTA_GEN_STAMP);
-      }
-      v.leaveEnclosingElement();
-    }
-  }
-  
-  /**
-   * Visit OP_RENAME_OLD
-   */
-  private void visit_OP_RENAME_OLD() throws IOException {
-    visitTxId();
-    if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
-      v.visitInt(EditsElement.LENGTH);
-    }
-    v.visitStringUTF8( EditsElement.SOURCE);
-    v.visitStringUTF8( EditsElement.DESTINATION);
-    if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
-      v.visitLong(EditsElement.TIMESTAMP);
-    } else {
-      v.visitStringUTF8(EditsElement.TIMESTAMP);
-    }
-  }
-
-  /**
-   * Visit OP_DELETE
-   */
-  private void visit_OP_DELETE() throws IOException {
-    visitTxId();
-    if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
-      v.visitInt(EditsElement.LENGTH);
-    }
-    v.visitStringUTF8( EditsElement.PATH);
-    if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
-      v.visitLong(EditsElement.TIMESTAMP);
-    } else {
-      v.visitStringUTF8(EditsElement.TIMESTAMP);
-    }
-  }
-
-  /**
-   * Visit OP_MKDIR
-   */
-  private void visit_OP_MKDIR() throws IOException {
-    visitTxId();
-    if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
-      v.visitInt(EditsElement.LENGTH);
-    }
-    v.visitStringUTF8( EditsElement.PATH);
-    if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
-      v.visitLong(EditsElement.TIMESTAMP);
-      v.visitLong(EditsElement.ATIME);
-    } else {
-      v.visitStringUTF8(EditsElement.TIMESTAMP);
-      v.visitStringUTF8(EditsElement.ATIME);
-    }
-    // PERMISSION_STATUS
-    v.visitEnclosingElement( EditsElement.PERMISSION_STATUS);
-
-    v.visitStringText( EditsElement.USERNAME);
-    v.visitStringText( EditsElement.GROUPNAME);
-    v.visitShort(      EditsElement.FS_PERMISSIONS);
-
-    v.leaveEnclosingElement();
-  }
-
-  /**
-   * Visit OP_SET_REPLICATION
-   */
-  private void visit_OP_SET_REPLICATION() throws IOException {
-    visitTxId();
-
-    v.visitStringUTF8(EditsElement.PATH);
-    if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
-      v.visitShort(EditsElement.REPLICATION);
-    } else {
-      v.visitStringUTF8(EditsElement.REPLICATION);
-    }
-  }
-
-  /**
-   * Visit OP_SET_PERMISSIONS
-   */
-  private void visit_OP_SET_PERMISSIONS() throws IOException {
-    visitTxId();
-
-    v.visitStringUTF8( EditsElement.PATH);
-    v.visitShort(      EditsElement.FS_PERMISSIONS);
-  }
-
-  /**
-   * Visit OP_SET_OWNER
-   */
-  private void visit_OP_SET_OWNER() throws IOException {
-    visitTxId();
-
-    v.visitStringUTF8(EditsElement.PATH);
-    v.visitStringUTF8(EditsElement.USERNAME);
-    v.visitStringUTF8(EditsElement.GROUPNAME);
-  }
-
-  /**
-   * Visit OP_SET_GENSTAMP
-   */
-  private void visit_OP_SET_GENSTAMP() throws IOException {
-    visitTxId();
-
-    v.visitLong(EditsElement.GENERATION_STAMP);
-  }
-
-  /**
-   * Visit OP_TIMES
-   */
-  private void visit_OP_TIMES() throws IOException {
-    visitTxId();
-    if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
-      v.visitInt(EditsElement.LENGTH);
-    }
-    v.visitStringUTF8( EditsElement.PATH);
-    if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
-      v.visitLong(EditsElement.MTIME);
-      v.visitLong(EditsElement.ATIME);
-    } else {
-      v.visitStringUTF8(EditsElement.MTIME);
-      v.visitStringUTF8(EditsElement.ATIME);
-    }
-  }
-
-  /**
-   * Visit OP_SET_QUOTA
-   */
-  private void visit_OP_SET_QUOTA() throws IOException {
-    visitTxId();
-
-    v.visitStringUTF8( EditsElement.PATH);
-    v.visitLong(       EditsElement.NS_QUOTA);
-    v.visitLong(       EditsElement.DS_QUOTA);
-  }
-
-  /**
-   * Visit OP_RENAME
-   */
-  private void visit_OP_RENAME() throws IOException {
-    visitTxId();
-    if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
-      v.visitInt(EditsElement.LENGTH);
-    }
-    v.visitStringUTF8(    EditsElement.SOURCE);
-    v.visitStringUTF8(    EditsElement.DESTINATION);
-    if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
-      v.visitLong(EditsElement.TIMESTAMP);
-    } else {
-      v.visitStringUTF8(EditsElement.TIMESTAMP);
-    }
-    v.visitBytesWritable( EditsElement.RENAME_OPTIONS);
-  }
-
-  /**
-   * Visit OP_CONCAT_DELETE
-   */
-  private void visit_OP_CONCAT_DELETE() throws IOException {
-    visitTxId();
-    int sourceCount = 0;
-    if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
-      IntToken lengthToken = v.visitInt(EditsElement.LENGTH);
-      sourceCount = lengthToken.value - 2;
-    }
-    v.visitStringUTF8(EditsElement.CONCAT_TARGET);
-    // all except of CONCAT_TARGET and TIMESTAMP
-    if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
-      IntToken lengthToken = v.visitInt(EditsElement.LENGTH);
-      sourceCount = lengthToken.value;
-    }
-    for(int i = 0; i < sourceCount; i++) {
-      v.visitStringUTF8(EditsElement.CONCAT_SOURCE);
-    }
-    if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
-      v.visitLong(EditsElement.TIMESTAMP);
-    } else {
-      v.visitStringUTF8(EditsElement.TIMESTAMP);
-    }
-  }
-
-  /**
-   * Visit OP_SYMLINK
-   */
-  private void visit_OP_SYMLINK() throws IOException {
-    visitTxId();
-    if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
-      v.visitInt(EditsElement.LENGTH);
-    }
-    v.visitStringUTF8( EditsElement.SOURCE);
-    v.visitStringUTF8( EditsElement.DESTINATION);
-    if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
-      v.visitLong(EditsElement.MTIME);
-      v.visitLong(EditsElement.ATIME);
-    } else {
-      v.visitStringUTF8(EditsElement.MTIME);
-      v.visitStringUTF8(EditsElement.ATIME);
-    }
-    // PERMISSION_STATUS
-    v.visitEnclosingElement(EditsElement.PERMISSION_STATUS);
-
-    v.visitStringText( EditsElement.USERNAME);
-    v.visitStringText( EditsElement.GROUPNAME);
-    v.visitShort(      EditsElement.FS_PERMISSIONS);
-
-    v.leaveEnclosingElement();
-  }
-
-  /**
-   * Visit OP_GET_DELEGATION_TOKEN
-   */
-  private void visit_OP_GET_DELEGATION_TOKEN() throws IOException {
-    visitTxId();
-    
-    v.visitByte(EditsElement.T_VERSION);
-    v.visitStringText(EditsElement.T_OWNER);
-    v.visitStringText(EditsElement.T_RENEWER);
-    v.visitStringText(EditsElement.T_REAL_USER);
-    v.visitVLong(EditsElement.T_ISSUE_DATE);
-    v.visitVLong(EditsElement.T_MAX_DATE);
-    v.visitVInt(EditsElement.T_SEQUENCE_NUMBER);
-    v.visitVInt(EditsElement.T_MASTER_KEY_ID);
-    if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
-      v.visitLong(EditsElement.T_EXPIRY_TIME);
-    } else {
-      v.visitStringUTF8(EditsElement.T_EXPIRY_TIME);
-    }
-  }
-
-  /**
-   * Visit OP_RENEW_DELEGATION_TOKEN
-   */
-  private void visit_OP_RENEW_DELEGATION_TOKEN()
-    throws IOException {
-    visitTxId();
-
-    v.visitByte(EditsElement.T_VERSION);
-    v.visitStringText(EditsElement.T_OWNER);
-    v.visitStringText(EditsElement.T_RENEWER);
-    v.visitStringText(EditsElement.T_REAL_USER);
-    v.visitVLong(EditsElement.T_ISSUE_DATE);
-    v.visitVLong(EditsElement.T_MAX_DATE);
-    v.visitVInt(EditsElement.T_SEQUENCE_NUMBER);
-    v.visitVInt(EditsElement.T_MASTER_KEY_ID);
-    if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
-      v.visitLong(EditsElement.T_EXPIRY_TIME);
-    } else {
-      v.visitStringUTF8(EditsElement.T_EXPIRY_TIME);
-    }
-  }
-
-  /**
-   * Visit OP_CANCEL_DELEGATION_TOKEN
-   */
-  private void visit_OP_CANCEL_DELEGATION_TOKEN()
-    throws IOException {
-    visitTxId();
-
-      v.visitByte(       EditsElement.T_VERSION);
-      v.visitStringText( EditsElement.T_OWNER);
-      v.visitStringText( EditsElement.T_RENEWER);
-      v.visitStringText( EditsElement.T_REAL_USER);
-      v.visitVLong(      EditsElement.T_ISSUE_DATE);
-      v.visitVLong(      EditsElement.T_MAX_DATE);
-      v.visitVInt(       EditsElement.T_SEQUENCE_NUMBER);
-      v.visitVInt(       EditsElement.T_MASTER_KEY_ID);
-  }
-
-  /**
-   * Visit OP_UPDATE_MASTER_KEY
-   */
-  private void visit_OP_UPDATE_MASTER_KEY()
-    throws IOException {
-    visitTxId();
-    
-      v.visitVInt(  EditsElement.KEY_ID);
-      v.visitVLong( EditsElement.KEY_EXPIRY_DATE);
-      VIntToken blobLengthToken = v.visitVInt(EditsElement.KEY_LENGTH);
-      v.visitBlob(EditsElement.KEY_BLOB, blobLengthToken.value);
-  }
-  
-  private void visit_OP_REASSIGN_LEASE()
-    throws IOException {
-    visitTxId();
-
-      v.visitStringUTF8(EditsElement.CLIENT_NAME);
-      v.visitStringUTF8(EditsElement.PATH);
-      v.visitStringUTF8(EditsElement.CLIENT_NAME);
-  }
-
-  /**
-   * Visit OP_BEGIN_LOG_SEGMENT
-   */
-  private void visit_OP_BEGIN_LOG_SEGMENT()
-    throws IOException {
-    visitTxId();
-  }
-  
-  /**
-   * Visit OP_END_LOG_SEGMENT
-   */
-  private void visit_OP_END_LOG_SEGMENT()
-    throws IOException {
-    visitTxId();
-  }
-
-  private void visitOpCode(FSEditLogOpCodes editsOpCode)
-    throws IOException {
-
-    switch(editsOpCode) {
-      case OP_INVALID: // -1
-        visit_OP_INVALID();
-        break;
-      case OP_ADD: // 0
-        visit_OP_ADD();
-        break;
-      case OP_CLOSE: // 9
-        visit_OP_CLOSE();
-        break;
-      case OP_RENAME_OLD: // 1
-        visit_OP_RENAME_OLD();
-        break;
-      case OP_DELETE: // 2
-        visit_OP_DELETE();
-        break;
-      case OP_MKDIR: // 3
-        visit_OP_MKDIR();
-        break;
-      case OP_SET_REPLICATION: // 4
-        visit_OP_SET_REPLICATION();
-        break;
-      case OP_SET_PERMISSIONS: // 7
-        visit_OP_SET_PERMISSIONS();
-        break;
-      case OP_SET_OWNER: // 8
-        visit_OP_SET_OWNER();
-        break;
-      case OP_SET_GENSTAMP: // 10
-        visit_OP_SET_GENSTAMP();
-        break;
-      case OP_TIMES: // 13
-        visit_OP_TIMES();
-        break;
-      case OP_SET_QUOTA: // 14
-        visit_OP_SET_QUOTA();
-        break;
-      case OP_RENAME: // 15
-        visit_OP_RENAME();
-        break;
-      case OP_CONCAT_DELETE: // 16
-        visit_OP_CONCAT_DELETE();
-        break;
-      case OP_SYMLINK: // 17
-        visit_OP_SYMLINK();
-        break;
-      case OP_GET_DELEGATION_TOKEN: // 18
-        visit_OP_GET_DELEGATION_TOKEN();
-        break;
-      case OP_RENEW_DELEGATION_TOKEN: // 19
-        visit_OP_RENEW_DELEGATION_TOKEN();
-        break;
-      case OP_CANCEL_DELEGATION_TOKEN: // 20
-        visit_OP_CANCEL_DELEGATION_TOKEN();
-        break;
-      case OP_UPDATE_MASTER_KEY: // 21
-        visit_OP_UPDATE_MASTER_KEY();
-        break;
-      case OP_REASSIGN_LEASE: // 22
-        visit_OP_REASSIGN_LEASE();
-        break;
-      case OP_END_LOG_SEGMENT: // 23
-        visit_OP_END_LOG_SEGMENT();
-        break;        
-      case OP_START_LOG_SEGMENT: // 24
-        visit_OP_BEGIN_LOG_SEGMENT();
-        break;
-      case OP_UPDATE_BLOCKS: // 25
-        visit_OP_UPDATE_BLOCKS();
-        break;
-      default:
-      {
-        throw new IOException("Unknown op code " + editsOpCode);
-      }
-    }
-  }
-
-  /**
-   * Loads edits file, uses visitor to process all elements
-   */
-  @Override
-  public void loadEdits() throws IOException {
-
-    try {
-      v.start();
-      v.visitEnclosingElement(EditsElement.EDITS);
-
-      IntToken editsVersionToken = v.visitInt(EditsElement.EDITS_VERSION);
-      editsVersion = editsVersionToken.value;
-      if(!canLoadVersion(editsVersion)) {
-        throw new IOException("Cannot process editLog version " +
-          editsVersionToken.value);
-      }
-
-      FSEditLogOpCodes editsOpCode;
-      do {
-        v.visitEnclosingElement(EditsElement.RECORD);
-
-        ByteToken opCodeToken;
-        try {
-          opCodeToken = v.visitByte(EditsElement.OPCODE);
-        } catch (EOFException eof) {
-          // Getting EOF when reading the opcode is fine --
-          // it's just a finalized edits file
-          // Just fake the OP_INVALID here.
-          opCodeToken = new ByteToken(EditsElement.OPCODE);
-          opCodeToken.fromByte(FSEditLogOpCodes.OP_INVALID.getOpCode());
-          v.visit(opCodeToken);
-        }
-        editsOpCode = FSEditLogOpCodes.fromByte(opCodeToken.value);
-
-        v.visitEnclosingElement(EditsElement.DATA);
-
-        visitOpCode(editsOpCode);
-
-        v.leaveEnclosingElement(); // DATA
-        
-        if (editsOpCode != FSEditLogOpCodes.OP_INVALID && 
-            LayoutVersion.supports(Feature.EDITS_CHESKUM, editsVersion)) {
-          v.visitInt(EditsElement.CHECKSUM);
-        }
-        v.leaveEnclosingElement(); // RECORD
-      } while(editsOpCode != FSEditLogOpCodes.OP_INVALID);
-
-      v.leaveEnclosingElement(); // EDITS
-      v.finish();
-    } catch(IOException e) {
-      // Tell the visitor to clean up, then re-throw the exception
-      v.finishAbnormally();
-      throw e;
-    }
-  }
-}

+ 0 - 175
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsVisitor.java

@@ -1,175 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.tools.offlineEditsViewer;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-import static org.apache.hadoop.hdfs.tools.offlineEditsViewer.Tokenizer.ByteToken;
-import static org.apache.hadoop.hdfs.tools.offlineEditsViewer.Tokenizer.ShortToken;
-import static org.apache.hadoop.hdfs.tools.offlineEditsViewer.Tokenizer.IntToken;
-import static org.apache.hadoop.hdfs.tools.offlineEditsViewer.Tokenizer.VIntToken;
-import static org.apache.hadoop.hdfs.tools.offlineEditsViewer.Tokenizer.LongToken;
-import static org.apache.hadoop.hdfs.tools.offlineEditsViewer.Tokenizer.VLongToken;
-import static org.apache.hadoop.hdfs.tools.offlineEditsViewer.Tokenizer.StringUTF8Token;
-import static org.apache.hadoop.hdfs.tools.offlineEditsViewer.Tokenizer.StringTextToken;
-import static org.apache.hadoop.hdfs.tools.offlineEditsViewer.Tokenizer.BlobToken;
-import static org.apache.hadoop.hdfs.tools.offlineEditsViewer.Tokenizer.BytesWritableToken;
-import static org.apache.hadoop.hdfs.tools.offlineEditsViewer.Tokenizer.EmptyToken;
-
-/**
- * An implementation of EditsVisitor can traverse the structure of an
- * Hadoop edits log and respond to each of the structures within the file.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-abstract public class EditsVisitor {
-
-  private Tokenizer tokenizer;
-
-  public EditsVisitor(Tokenizer tokenizer) {
-    this.tokenizer = tokenizer;
-  }
-
-  /**
-   * Begin visiting the edits log structure.  Opportunity to perform
-   * any initialization necessary for the implementing visitor.
-   */
-  abstract void start() throws IOException;
-
-  /**
-   * Finish visiting the edits log structure.  Opportunity to perform any
-   * clean up necessary for the implementing visitor.
-   */
-  abstract void finish() throws IOException;
-
-  /**
-   * Finish visiting the edits log structure after an error has occurred
-   * during the processing.  Opportunity to perform any clean up necessary
-   * for the implementing visitor.
-   */
-  abstract void finishAbnormally() throws IOException;
-
-  /**
-   * Visit non enclosing element of edits log with specified value.
-   *
-   * @param value a token to visit
-   */
-  abstract Tokenizer.Token visit(Tokenizer.Token value) throws IOException;
-
-  /**
-   * Convenience shortcut method to parse a specific token type
-   */
-  public ByteToken visitByte(EditsElement e) throws IOException {
-    return (ByteToken)visit(tokenizer.read(new ByteToken(e)));
-  }
-
-  /**
-   * Convenience shortcut method to parse a specific token type
-   */
-  public ShortToken visitShort(EditsElement e) throws IOException {
-    return (ShortToken)visit(tokenizer.read(new ShortToken(e)));
-  }
-
-  /**
-   * Convenience shortcut method to parse a specific token type
-   */
-  public IntToken visitInt(EditsElement e) throws IOException {
-    return (IntToken)visit(tokenizer.read(new IntToken(e)));
-  }
-
-  /**
-   * Convenience shortcut method to parse a specific token type
-   */
-  public VIntToken visitVInt(EditsElement e) throws IOException {
-    return (VIntToken)visit(tokenizer.read(new VIntToken(e)));
-  }
-
-  /**
-   * Convenience shortcut method to parse a specific token type
-   */
-  public LongToken visitLong(EditsElement e) throws IOException {
-    return (LongToken)visit(tokenizer.read(new LongToken(e)));
-  }
-
-  /**
-   * Convenience shortcut method to parse a specific token type
-   */
-  public VLongToken visitVLong(EditsElement e) throws IOException {
-    return (VLongToken)visit(tokenizer.read(new VLongToken(e)));
-  }
-
-  /**
-   * Convenience shortcut method to parse a specific token type
-   */
-  public StringUTF8Token visitStringUTF8(EditsElement e) throws IOException {
-    return (StringUTF8Token)visit(tokenizer.read(new StringUTF8Token(e)));
-  }
-
-  /**
-   * Convenience shortcut method to parse a specific token type
-   */
-  public StringTextToken visitStringText(EditsElement e) throws IOException {
-    return (StringTextToken)visit(tokenizer.read(new StringTextToken(e)));
-  }
-
-  /**
-   * Convenience shortcut method to parse a specific token type
-   */
-  public BlobToken visitBlob(EditsElement e, int length) throws IOException {
-    return (BlobToken)visit(tokenizer.read(new BlobToken(e, length)));
-  }
-
-  /**
-   * Convenience shortcut method to parse a specific token type
-   */
-  public BytesWritableToken visitBytesWritable(EditsElement e) throws IOException {
-    return (BytesWritableToken)visit(tokenizer.read(new BytesWritableToken(e)));
-  }
-
-  /**
-   * Convenience shortcut method to parse a specific token type
-   */
-  public EmptyToken visitEmpty(EditsElement e) throws IOException {
-    return (EmptyToken)visit(tokenizer.read(new EmptyToken(e)));
-  }
-
-  /**
-   * Begin visiting an element that encloses another element, such as
-   * the beginning of the list of blocks that comprise a file.
-   *
-   * @param value Token being visited
-   */
-  abstract void visitEnclosingElement(Tokenizer.Token value)
-     throws IOException;
-
-  /**
-   * Convenience shortcut method (it virutally always uses EmptyToken)
-   */
-  void visitEnclosingElement(EditsElement e) throws IOException {
-    visitEnclosingElement(tokenizer.read(new EmptyToken(e)));
-  }
-
-  /**
-   * Leave current enclosing element.  Called, for instance, at the end of
-   * processing the blocks that compromise a file.
-   */
-  abstract void leaveEnclosingElement() throws IOException;
-}

+ 0 - 55
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsVisitorFactory.java

@@ -1,55 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.tools.offlineEditsViewer;
-
-import java.io.IOException;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * EditsVisitorFactory for different implementations of EditsVisitor
- *
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class EditsVisitorFactory {
-
-  /**
-   * Factory function that creates an EditsVisitor object
-   *
-   * @param filename output filename
-   * @param tokenizer input tokenizer
-   * @return EditsVisitor for appropriate output format (binary, XML etc.)
-   */
-  static public EditsVisitor getEditsVisitor(String filename,
-    String processor,
-    Tokenizer tokenizer,
-    boolean printToScreen) throws IOException {
-
-    if(processor.toLowerCase().equals("xml")) {
-      return new XmlEditsVisitor(filename, tokenizer, printToScreen);
-    } else if(processor.toLowerCase().equals("stats")) {
-      return new StatisticsEditsVisitor(filename, tokenizer, printToScreen);
-    } else if(processor.toLowerCase().equals("binary")) {
-      return new BinaryEditsVisitor(filename, tokenizer, printToScreen);
-    } else {
-      throw new IOException("Unknown proccesor " + processor +
-        " (valid processors: xml, binary, stats)");
-    }
-  }
-}

+ 83 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsBinaryLoader.java

@@ -0,0 +1,83 @@
+/**
+ * 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.tools.offlineEditsViewer;
+
+import java.io.IOException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
+
+import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream;
+
+/**
+ * OfflineEditsBinaryLoader loads edits from a binary edits file
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+class OfflineEditsBinaryLoader implements OfflineEditsLoader {
+  private OfflineEditsVisitor visitor;
+  private EditLogInputStream inputStream;
+  private boolean fixTxIds;
+  private long nextTxId;
+  
+  /**
+   * Constructor
+   */
+  public OfflineEditsBinaryLoader(OfflineEditsVisitor visitor,
+        EditLogInputStream inputStream) {
+    this.visitor = visitor;
+    this.inputStream = inputStream;
+    this.fixTxIds = false;
+    this.nextTxId = -1;
+  }
+
+  /**
+   * Loads edits file, uses visitor to process all elements
+   */
+  public void loadEdits() throws IOException {
+    try {
+      visitor.start(inputStream.getVersion());
+      while (true) {
+        FSEditLogOp op = inputStream.readOp();
+        if (op == null)
+          break;
+        if (fixTxIds) {
+          if (nextTxId <= 0) {
+            nextTxId = op.getTransactionId();
+            if (nextTxId <= 0) {
+              nextTxId = 1;
+            }
+          }
+          op.setTransactionId(nextTxId);
+          nextTxId++;
+        }
+        visitor.visitOp(op);
+      }
+      visitor.close(null);
+    } catch(IOException e) {
+      // Tell the visitor to clean up, then re-throw the exception
+      visitor.close(e);
+      throw e;
+    }
+  }
+  
+  public void setFixTxIds() {
+    fixTxIds = true;
+  }
+}

+ 62 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsLoader.java

@@ -0,0 +1,62 @@
+/**
+ * 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.tools.offlineEditsViewer;
+
+import java.io.File;
+import java.io.IOException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import org.apache.hadoop.hdfs.server.namenode.EditLogFileInputStream;
+
+import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream;
+
+/**
+ * OfflineEditsLoader walks an EditsVisitor over an EditLogInputStream
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+interface OfflineEditsLoader {
+  
+  abstract public void loadEdits() throws IOException;
+  
+  public abstract void setFixTxIds();
+  
+  static class OfflineEditsLoaderFactory {
+    static OfflineEditsLoader createLoader(OfflineEditsVisitor visitor,
+        String inputFileName, boolean xmlInput) throws IOException {
+      if (xmlInput) {
+        return new OfflineEditsXmlLoader(visitor, new File(inputFileName));
+      } else {
+        File file = null;
+        EditLogInputStream elis = null;
+        OfflineEditsLoader loader = null;
+        try {
+          file = new File(inputFileName);
+          elis = new EditLogFileInputStream(file, -1, -1, false);
+          loader = new OfflineEditsBinaryLoader(visitor, elis);
+        } finally {
+          if ((loader == null) && (elis != null)) {
+            elis.close();
+          }
+        }
+        return loader;
+      }
+    }
+  }
+}

+ 54 - 50
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsViewer.java

@@ -18,12 +18,16 @@
 package org.apache.hadoop.hdfs.tools.offlineEditsViewer;
 
 import java.io.EOFException;
+import java.io.File;
 import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 
 import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.hdfs.server.namenode.EditLogFileInputStream;
+import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream;
+import org.apache.hadoop.hdfs.tools.offlineEditsViewer.OfflineEditsLoader.OfflineEditsLoaderFactory;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 
@@ -33,6 +37,7 @@ import org.apache.commons.cli.OptionBuilder;
 import org.apache.commons.cli.Options;
 import org.apache.commons.cli.ParseException;
 import org.apache.commons.cli.PosixParser;
+import org.xml.sax.SAXParseException;
 
 /**
  * This class implements an offline edits viewer, tool that
@@ -42,28 +47,8 @@ import org.apache.commons.cli.PosixParser;
 @InterfaceStability.Unstable
 public class OfflineEditsViewer extends Configured implements Tool {
 
-  private EditsLoader  editsLoader;
   private final static String defaultProcessor = "xml";
 
-  /**
-   * Set editsLoader
-   *
-   * @param editsLoader EditsLoader
-   */
-  private void setEditsLoader(EditsLoader editsLoader) {
-    this.editsLoader = editsLoader;
-  }
-
-  /**
-   * Process EditLog file.
-   *
-   * @param visitor use this visitor to process the file
-   */
-  public void go(EditsVisitor visitor) throws IOException  {
-    setEditsLoader(EditsLoader.LoaderFactory.getLoader(visitor));
-    editsLoader.loadEdits();
-  }
-
   /**
    * Print help.
    */  
@@ -90,6 +75,9 @@ public class OfflineEditsViewer extends Configured implements Tool {
       "                       format), stats (prints statistics about\n" +
       "                       edits file)\n" +
       "-h,--help              Display usage information and exit\n" +
+      "-f,--fix-txids         Renumber the transaction IDs in the input,\n" +
+      "                       so that there are no gaps or invalid " +
+      "                       transaction IDs.\n" +
       "-v,--verbose           More verbose output, prints the input and\n" +
       "                       output filenames, for processors that write\n" +
       "                       to a file, also output to screen. On large\n" +
@@ -124,11 +112,48 @@ public class OfflineEditsViewer extends Configured implements Tool {
     
     options.addOption("p", "processor", true, "");
     options.addOption("v", "verbose", false, "");
+    options.addOption("f", "fix-txids", false, "");
     options.addOption("h", "help", false, "");
 
     return options;
   }
 
+  /** Process an edit log using the chosen processor or visitor.
+   * 
+   * @param inputFilename   The file to process
+   * @param outputFilename  The output file name
+   * @param processor       If visitor is null, the processor to use
+   * @param visitor         If non-null, the visitor to use.
+   * 
+   * @return                0 on success; error code otherwise
+   */
+  public int go(String inputFileName, String outputFileName, String processor,
+      boolean printToScreen, boolean fixTxIds, OfflineEditsVisitor visitor)
+  {
+    if (printToScreen) {
+      System.out.println("input  [" + inputFileName  + "]");
+      System.out.println("output [" + outputFileName + "]");
+    }
+    try {
+      if (visitor == null) {
+        visitor = OfflineEditsVisitorFactory.getEditsVisitor(
+            outputFileName, processor, printToScreen);
+      }
+      boolean xmlInput = inputFileName.endsWith(".xml");
+      OfflineEditsLoader loader = OfflineEditsLoaderFactory.
+          createLoader(visitor, inputFileName, xmlInput);
+      if (fixTxIds) {
+        loader.setFixTxIds();
+      }
+      loader.loadEdits();
+    } catch(Exception e) {
+      System.err.println("Encountered exception. Exiting: " + e.getMessage());
+      e.printStackTrace(System.err);
+      return -1;
+    }
+    return 0;
+  }
+
   /**
    * Main entry point for ToolRunner (see ToolRunner docs)
    *
@@ -137,17 +162,13 @@ public class OfflineEditsViewer extends Configured implements Tool {
    */
   @Override
   public int run(String[] argv) throws Exception {
-    int exitCode = 0;
-
     Options options = buildOptions();
     if(argv.length == 0) {
       printHelp();
       return -1;
     }
-
     CommandLineParser parser = new PosixParser();
     CommandLine cmd;
-
     try {
       cmd = parser.parse(options, argv);
     } catch (ParseException e) {
@@ -156,37 +177,20 @@ public class OfflineEditsViewer extends Configured implements Tool {
       printHelp();
       return -1;
     }
-
     if(cmd.hasOption("h")) { // print help and exit
       printHelp();
       return -1;
     }
-
-    boolean printToScreen    = false;
-    String inputFilenameArg  = cmd.getOptionValue("i");
-    String outputFilenameArg = cmd.getOptionValue("o");
-    String processor         = cmd.getOptionValue("p");
-    if(processor == null) { processor = defaultProcessor; }
-
-    if(cmd.hasOption("v")) { // print output to screen too
-      printToScreen = true;
-      System.out.println("input  [" + inputFilenameArg  + "]");
-      System.out.println("output [" + outputFilenameArg + "]");
+    String inputFileName = cmd.getOptionValue("i");
+    String outputFileName = cmd.getOptionValue("o");
+    String processor = cmd.getOptionValue("p");
+    if(processor == null) {
+      processor = defaultProcessor;
     }
-
-    try {
-      go(EditsVisitorFactory.getEditsVisitor(
-        outputFilenameArg,
-        processor,
-        TokenizerFactory.getTokenizer(inputFilenameArg),
-        printToScreen));
-    } catch (EOFException e) {
-      System.err.println("Input file ended unexpectedly. Exiting");
-    } catch(IOException e) {
-      System.err.println("Encountered exception. Exiting: " + e.getMessage());
-    }
-
-    return exitCode;
+    boolean printToScreen = cmd.hasOption("v");
+    boolean fixTxIds = cmd.hasOption("f");
+    return go(inputFileName, outputFileName, processor,
+        printToScreen, fixTxIds, null);
   }
 
   /**

+ 21 - 29
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/BinaryTokenizer.java → hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsVisitor.java

@@ -17,51 +17,43 @@
  */
 package org.apache.hadoop.hdfs.tools.offlineEditsViewer;
 
-import java.io.FileInputStream;
-import java.io.DataInputStream;
-import java.io.FileNotFoundException;
 import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
 
 /**
- * Tokenizer that reads tokens from a binary file
- *
+ * An implementation of OfflineEditsVisitor can traverse the structure of an
+ * Hadoop edits log and respond to each of the structures within the file.
  */
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
-public class BinaryTokenizer implements Tokenizer {
-
-  private DataInputStream in;
-
+abstract public interface OfflineEditsVisitor {
   /**
-   * BinaryTokenizer constructor
-   *
-   * @param filename input filename
+   * Begin visiting the edits log structure.  Opportunity to perform
+   * any initialization necessary for the implementing visitor.
+   * 
+   * @param version     Edit log version
    */
-  public BinaryTokenizer(String filename) throws FileNotFoundException {
-    in = new DataInputStream(new FileInputStream(filename));
-  }
+  abstract void start(int version) throws IOException;
 
   /**
-   * BinaryTokenizer constructor
-   *
-   * @param in input stream
+   * Finish visiting the edits log structure.  Opportunity to perform any
+   * clean up necessary for the implementing visitor.
+   * 
+   * @param error        If the visitor was closed because of an 
+   *                     unrecoverable error in the input stream, this 
+   *                     is the exception.
    */
-  public BinaryTokenizer(DataInputStream in) throws IOException {
-    this.in = in;
-  }
+  abstract void close(Throwable error) throws IOException;
 
   /**
-   * @see org.apache.hadoop.hdfs.tools.offlineEditsViewer.Tokenizer#read
+   * Begin visiting an element that encloses another element, such as
+   * the beginning of the list of blocks that comprise a file.
    *
-   * @param t a Token to read
-   * @return token that was just read
+   * @param value Token being visited
    */
-  @Override
-  public Token read(Token t) throws IOException {
-    t.fromBinary(in);
-    return t;
-  }
+  abstract void visitOp(FSEditLogOp op)
+     throws IOException;
 }

+ 77 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsVisitorFactory.java

@@ -0,0 +1,77 @@
+/**
+ * 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.tools.offlineEditsViewer;
+
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.io.IOUtils;
+
+/**
+ * EditsVisitorFactory for different implementations of EditsVisitor
+ *
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class OfflineEditsVisitorFactory {
+  /**
+   * Factory function that creates an EditsVisitor object
+   *
+   * @param filename              output filename
+   * @param processor             type of visitor to create 
+   * @param printToScreen         parameter passed to visitor constructor
+   *
+   * @return EditsVisitor for appropriate output format (binary, xml, etc.)
+   */
+  static public OfflineEditsVisitor getEditsVisitor(String filename,
+    String processor, boolean printToScreen) throws IOException {
+    if(processor.toLowerCase().equals("binary")) {
+      return new BinaryEditsVisitor(filename);
+    }
+    OfflineEditsVisitor vis;
+    OutputStream fout = new FileOutputStream(filename);
+    OutputStream out = null;
+    try {
+      if (!printToScreen) {
+        out = fout;
+      }
+      else {
+        OutputStream outs[] = new OutputStream[2];
+        outs[0] = fout;
+        outs[1] = System.out;
+        out = new TeeOutputStream(outs);
+      }
+      if(processor.toLowerCase().equals("xml")) {
+        vis = new XmlEditsVisitor(out);
+      } else if(processor.toLowerCase().equals("stats")) {
+        vis = new StatisticsEditsVisitor(out);
+      } else {
+        throw new IOException("Unknown proccesor " + processor +
+          " (valid processors: xml, binary, stats)");
+      }
+      out = fout = null;
+      return vis;
+    } finally {
+      IOUtils.closeStream(fout);
+      IOUtils.closeStream(out);
+    }
+  }
+}

+ 256 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsXmlLoader.java

@@ -0,0 +1,256 @@
+/**
+ * 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.tools.offlineEditsViewer;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.Stack;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.util.XMLUtils.InvalidXmlException;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes;
+
+import org.apache.hadoop.hdfs.util.XMLUtils.Stanza;
+import org.xml.sax.Attributes;
+import org.xml.sax.InputSource;
+import org.xml.sax.SAXException;
+import org.xml.sax.SAXParseException;
+import org.xml.sax.XMLReader;
+import org.xml.sax.helpers.DefaultHandler;
+import org.xml.sax.helpers.XMLReaderFactory;
+
+/**
+ * OfflineEditsXmlLoader walks an EditsVisitor over an OEV XML file
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+class OfflineEditsXmlLoader 
+    extends DefaultHandler implements OfflineEditsLoader {
+  private boolean fixTxIds;
+  private OfflineEditsVisitor visitor;
+  private FileReader fileReader;
+  private ParseState state;
+  private Stanza stanza;
+  private Stack<Stanza> stanzaStack;
+  private FSEditLogOpCodes opCode;
+  private StringBuffer cbuf;
+  private long nextTxId;
+  
+  static enum ParseState {
+    EXPECT_EDITS_TAG,
+    EXPECT_VERSION,
+    EXPECT_RECORD,
+    EXPECT_OPCODE,
+    EXPECT_DATA,
+    HANDLE_DATA,
+    EXPECT_END,
+  }
+  
+  public OfflineEditsXmlLoader(OfflineEditsVisitor visitor,
+        File inputFile) throws FileNotFoundException {
+    this.visitor = visitor;
+    this.fileReader = new FileReader(inputFile);
+  }
+
+  /**
+   * Loads edits file, uses visitor to process all elements
+   */
+  public void loadEdits() throws IOException {
+    try {
+      XMLReader xr = XMLReaderFactory.createXMLReader();
+      xr.setContentHandler(this);
+      xr.setErrorHandler(this);
+      xr.setDTDHandler(null);
+      xr.parse(new InputSource(fileReader));
+      visitor.close(null);
+    } catch (SAXParseException e) {
+      System.out.println("XML parsing error: " + "\n" +
+          "Line:    " + e.getLineNumber() + "\n" +
+          "URI:     " + e.getSystemId() + "\n" +
+          "Message: " + e.getMessage());        
+      visitor.close(e);
+      throw new IOException(e.toString());
+    } catch (SAXException e) {
+      visitor.close(e);
+      throw new IOException(e.toString());
+    } catch (RuntimeException e) {
+      visitor.close(e);
+      throw e;
+    } finally {
+      fileReader.close();
+    }
+  }
+  
+  @Override
+  public void startDocument() {
+    state = ParseState.EXPECT_EDITS_TAG;
+    stanza = null;
+    stanzaStack = new Stack<Stanza>();
+    opCode = null;
+    cbuf = new StringBuffer();
+    nextTxId = -1;
+  }
+  
+  @Override
+  public void endDocument() {
+    if (state != ParseState.EXPECT_END) {
+      throw new InvalidXmlException("expecting </EDITS>");
+    }
+  }
+  
+  public void startElement (String uri, String name,
+      String qName, Attributes atts) {
+    switch (state) {
+    case EXPECT_EDITS_TAG:
+      if (!name.equals("EDITS")) {
+        throw new InvalidXmlException("you must put " +
+            "<EDITS> at the top of the XML file! " +
+            "Got tag " + name + " instead");
+      }
+      state = ParseState.EXPECT_VERSION;
+      break;
+    case EXPECT_VERSION:
+      if (!name.equals("EDITS_VERSION")) {
+        throw new InvalidXmlException("you must put " +
+            "<EDITS_VERSION> at the top of the XML file! " +
+            "Got tag " + name + " instead");
+      }
+      break;
+    case EXPECT_RECORD:
+      if (!name.equals("RECORD")) {
+        throw new InvalidXmlException("expected a <RECORD> tag");
+      }
+      state = ParseState.EXPECT_OPCODE;
+      break;
+    case EXPECT_OPCODE:
+      if (!name.equals("OPCODE")) {
+        throw new InvalidXmlException("expected an <OPCODE> tag");
+      }
+      break;
+    case EXPECT_DATA:
+      if (!name.equals("DATA")) {
+        throw new InvalidXmlException("expected a <DATA> tag");
+      }
+      stanza = new Stanza();
+      state = ParseState.HANDLE_DATA;
+      break;
+    case HANDLE_DATA:
+      Stanza parent = stanza;
+      Stanza child = new Stanza();
+      stanzaStack.push(parent);
+      stanza = child;
+      parent.addChild(name, child);
+      break;
+    case EXPECT_END:
+      throw new InvalidXmlException("not expecting anything after </EDITS>");
+    }
+  }
+  
+  public void endElement (String uri, String name, String qName) {
+    String str = cbuf.toString().trim();
+    cbuf = new StringBuffer();
+    switch (state) {
+    case EXPECT_EDITS_TAG:
+      throw new InvalidXmlException("expected <EDITS/>");
+    case EXPECT_VERSION:
+      if (!name.equals("EDITS_VERSION")) {
+        throw new InvalidXmlException("expected </EDITS_VERSION>");
+      }
+      try {
+        int version = Integer.parseInt(str);
+        visitor.start(version);
+      } catch (IOException e) {
+        // Can't throw IOException from a SAX method, sigh.
+        throw new RuntimeException(e);
+      }
+      state = ParseState.EXPECT_RECORD;
+      break;
+    case EXPECT_RECORD:
+      if (name.equals("EDITS")) {
+        state = ParseState.EXPECT_END;
+      } else if (!name.equals("RECORD")) {
+        throw new InvalidXmlException("expected </EDITS> or </RECORD>");
+      }
+      break;
+    case EXPECT_OPCODE:
+      if (!name.equals("OPCODE")) {
+        throw new InvalidXmlException("expected </OPCODE>");
+      }
+      opCode = FSEditLogOpCodes.valueOf(str);
+      state = ParseState.EXPECT_DATA;
+      break;
+    case EXPECT_DATA:
+      throw new InvalidXmlException("expected <DATA/>");
+    case HANDLE_DATA:
+      stanza.setValue(str);
+      if (stanzaStack.empty()) {
+        if (!name.equals("DATA")) {
+          throw new InvalidXmlException("expected </DATA>");
+        }
+        state = ParseState.EXPECT_RECORD;
+        FSEditLogOp op = FSEditLogOp.getOpInstance(opCode);
+        opCode = null;
+        try {
+          op.decodeXml(stanza);
+          stanza = null;
+        } finally {
+          if (stanza != null) {
+            System.err.println("fromXml error decoding opcode " + opCode +
+                "\n" + stanza.toString());
+            stanza = null;
+          }
+        }
+        if (fixTxIds) {
+          if (nextTxId <= 0) {
+            nextTxId = op.getTransactionId();
+            if (nextTxId <= 0) {
+              nextTxId = 1;
+            }
+          }
+          op.setTransactionId(nextTxId);
+          nextTxId++;
+        }
+        try {
+          visitor.visitOp(op);
+        } catch (IOException e) {
+          // Can't throw IOException from a SAX method, sigh.
+          throw new RuntimeException(e);
+        }
+        state = ParseState.EXPECT_RECORD;
+      } else {
+        stanza = stanzaStack.pop();
+      }
+      break;
+    case EXPECT_END:
+      throw new InvalidXmlException("not expecting anything after </EDITS>");
+    }
+  }
+  
+  public void characters (char ch[], int start, int length) {
+    cbuf.append(ch, start, length);
+  }
+
+  @Override
+  public void setFixTxIds() {
+    fixTxIds = true;
+  }
+}

+ 26 - 106
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/StatisticsEditsVisitor.java

@@ -19,12 +19,15 @@ package org.apache.hadoop.hdfs.tools.offlineEditsViewer;
 
 import java.io.FileWriter;
 import java.io.IOException;
+import java.io.OutputStream;
+import java.io.PrintStream;
 import java.util.Map;
 import java.util.HashMap;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes;
 
 /**
@@ -34,25 +37,13 @@ import org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes;
  */
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
-public class StatisticsEditsVisitor extends EditsVisitor {
-  private boolean printToScreen = false;
-  private boolean okToWrite = false;
-  final private FileWriter fw;
+public class StatisticsEditsVisitor implements OfflineEditsVisitor {
+  final private PrintStream out;
 
-  public final Map<FSEditLogOpCodes, Long> opCodeCount =
+  private int version = -1;
+  private final Map<FSEditLogOpCodes, Long> opCodeCount =
     new HashMap<FSEditLogOpCodes, Long>();
 
-  /**
-   * Create a processor that writes to the file named.
-   *
-   * @param filename Name of file to write output to
-   */
-  public StatisticsEditsVisitor(String filename, Tokenizer tokenizer)
-    throws IOException {
-
-    this(filename, tokenizer, false);
-  }
-
   /**
    * Create a processor that writes to the file named and may or may not
    * also output to the screen, as specified.
@@ -61,103 +52,29 @@ public class StatisticsEditsVisitor extends EditsVisitor {
    * @param tokenizer Input tokenizer
    * @param printToScreen Mirror output to screen?
    */
-  public StatisticsEditsVisitor(String filename,
-    Tokenizer tokenizer,
-    boolean printToScreen) throws IOException {
-
-    super(tokenizer);
-    this.printToScreen = printToScreen;
-    fw = new FileWriter(filename);
-    okToWrite = true;
+  public StatisticsEditsVisitor(OutputStream out) throws IOException {
+    this.out = new PrintStream(out);
   }
 
-  /**
-   * Start the visitor (initialization)
-   */
+  /** Start the visitor */
   @Override
-  void start() throws IOException {
-    // nothing to do
+  public void start(int version) throws IOException {
+    this.version = version;
   }
   
-  /* (non-Javadoc)
-   * @see org.apache.hadoop.hdfs.tools.offlineEditsViewer.EditsVisitor#finish()
-   */
+  /** Close the visitor */
   @Override
-  void finish() throws IOException {
-    write(getStatisticsString());
-    close();
-  }
-
-  /* (non-Javadoc)
-   * @see org.apache.hadoop.hdfs.tools.offlineEditsViewer.EditsVisitor#finishAbnormally()
-   */
-  @Override
-  void finishAbnormally() throws IOException {
-    close();
-  }
-
-  /**
-   * Close output stream and prevent further writing
-   */
-  private void close() throws IOException {
-    fw.close();
-    okToWrite = false;
-  }
-
-  /**
-   * Visit a enclosing element (element that has other elements in it)
-   */
-  @Override
-  void visitEnclosingElement(Tokenizer.Token value) throws IOException {
-    // nothing to do
-  }
-
-  /**
-   * End of eclosing element
-   */
-  @Override
-  void leaveEnclosingElement() throws IOException {
-    // nothing to do
-  }  
-
-  /**
-   * Visit a Token, calculate statistics
-   *
-   * @param value a Token to visit
-   */
-  @Override
-  Tokenizer.Token visit(Tokenizer.Token value) throws IOException {
-    // count the opCodes
-    if(value.getEditsElement() == EditsElement.OPCODE) {
-      if(value instanceof Tokenizer.ByteToken) {
-        incrementOpCodeCount(
-          FSEditLogOpCodes.fromByte(((Tokenizer.ByteToken)value).value));
-      } else {
-        throw new IOException("Token for EditsElement.OPCODE should be " +
-          "of type Tokenizer.ByteToken, not " + value.getClass());
-      }
+  public void close(Throwable error) throws IOException {
+    out.print(getStatisticsString());
+    if (error != null) {
+      out.print("EXITING ON ERROR: " + error.toString() + "\n");
     }
-    return value;
+    out.close();
   }
 
-  /**
-   * Write parameter to output file (and possibly screen).
-   *
-   * @param toWrite Text to write to file
-   */
-  protected void write(String toWrite) throws IOException  {
-    if(!okToWrite)
-      throw new IOException("file not open for writing.");
-
-    if(printToScreen)
-      System.out.print(toWrite);
-
-    try {
-      fw.write(toWrite);
-    } catch (IOException e) {
-      okToWrite = false;
-      throw e;
-    }
+  @Override
+  public void visitOp(FSEditLogOp op) throws IOException {
+    incrementOpCodeCount(op.opCode);
   }
 
   /**
@@ -189,13 +106,16 @@ public class StatisticsEditsVisitor extends EditsVisitor {
    */
   public String getStatisticsString() {
     StringBuffer sb = new StringBuffer();
+    sb.append(String.format(
+        "    %-30.30s      : %d%n",
+        "VERSION", version));
     for(FSEditLogOpCodes opCode : FSEditLogOpCodes.values()) {
       sb.append(String.format(
         "    %-30.30s (%3d): %d%n",
-        opCode,
+        opCode.toString(),
         opCode.getOpCode(),
         opCodeCount.get(opCode)));
     }
     return sb.toString();
   }
-}
+}

+ 40 - 19
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TokenizerFactory.java → hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TeeOutputStream.java

@@ -18,29 +18,50 @@
 package org.apache.hadoop.hdfs.tools.offlineEditsViewer;
 
 import java.io.IOException;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
+import java.io.OutputStream;
 
 /**
- * TokenizerFactory for different implementations of Tokenizer
- *
+ * A TeeOutputStream writes its output to multiple output streams.
  */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class TokenizerFactory {
-
-  /**
-   * Factory function that creates a Tokenizer object, the input format
-   * is set based on filename (*.xml is XML, otherwise binary)
-   *
-   * @param filename input filename
-   */
-  static public Tokenizer getTokenizer(String filename) throws IOException {
-    if(filename.toLowerCase().endsWith("xml")) {
-      return new XmlTokenizer(filename);
-    } else {
-      return new BinaryTokenizer(filename);
+public class TeeOutputStream extends OutputStream {
+  private OutputStream outs[];
+
+  public TeeOutputStream(OutputStream outs[]) {
+    this.outs = outs;
+  }
+
+  @Override
+  public void write(int c) throws IOException {
+    for (OutputStream o : outs) {
+     o.write(c);
     }
   }
 
+  @Override
+  public void write(byte[] b) throws IOException {
+    for (OutputStream o : outs) {
+     o.write(b);
+    }
+  }
+
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    for (OutputStream o : outs) {
+     o.write(b, off, len);
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    for (OutputStream o : outs) {
+     o.close();
+    }
+  }
+
+  @Override
+  public void flush() throws IOException {
+    for (OutputStream o : outs) {
+     o.flush();
+    }
+  }
 }

+ 0 - 109
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TextEditsVisitor.java

@@ -1,109 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.tools.offlineEditsViewer;
-
-import java.io.FileWriter;
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * TextEditsVisitor implements text version of EditsVisitor
- *
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-abstract class TextEditsVisitor extends EditsVisitor {
-  private boolean printToScreen = false;
-  private boolean okToWrite = false;
-  final private FileWriter fw;
-
-  /**
-   * Create a processor that writes to the file named.
-   *
-   * @param filename Name of file to write output to
-   */
-  public TextEditsVisitor(String filename, Tokenizer tokenizer)
-    throws IOException {
-
-    this(filename, tokenizer, false);
-  }
-
-  /**
-   * Create a processor that writes to the file named and may or may not
-   * also output to the screen, as specified.
-   *
-   * @param filename Name of file to write output to
-   * @param tokenizer Input tokenizer
-   * @param printToScreen Mirror output to screen?
-   */
-  public TextEditsVisitor(String filename,
-    Tokenizer tokenizer,
-    boolean printToScreen) throws IOException {
-
-    super(tokenizer);
-    this.printToScreen = printToScreen;
-    fw = new FileWriter(filename);
-    okToWrite = true;
-  }
-  
-  /* (non-Javadoc)
-   * @see org.apache.hadoop.hdfs.tools.offlineEditsViewer.EditsVisitor#finish()
-   */
-  @Override
-  void finish() throws IOException {
-    close();
-  }
-
-  /* (non-Javadoc)
-   * @see org.apache.hadoop.hdfs.tools.offlineEditsViewer.EditsVisitor#finishAbnormally()
-   */
-  @Override
-  void finishAbnormally() throws IOException {
-    close();
-  }
-
-  /**
-   * Close output stream and prevent further writing
-   */
-  private void close() throws IOException {
-    fw.close();
-    okToWrite = false;
-  }
-
-  /**
-   * Write parameter to output file (and possibly screen).
-   *
-   * @param toWrite Text to write to file
-   */
-  protected void write(String toWrite) throws IOException  {
-    if(!okToWrite)
-      throw new IOException("file not open for writing.");
-
-    if(printToScreen)
-      System.out.print(toWrite);
-
-    try {
-      fw.write(toWrite);
-    } catch (IOException e) {
-      okToWrite = false;
-      throw e;
-    }
-  }
-}

+ 0 - 401
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/Tokenizer.java

@@ -1,401 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.tools.offlineEditsViewer;
-
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-import org.apache.commons.codec.binary.Base64;
-
-import org.apache.hadoop.hdfs.DeprecatedUTF8;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.WritableUtils;
-import org.apache.hadoop.io.BytesWritable;
-
-import org.apache.hadoop.hdfs.tools.offlineEditsViewer.EditsElement;
-
-/**
- * Tokenizer that hides the details of different input formats
- *
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-interface Tokenizer {
-
-  /**
-   * Abstract class Token, derive Tokens of needed types from
-   * this class
-   */
-  @InterfaceAudience.Private
-  @InterfaceStability.Unstable
-  abstract public class Token {
-    EditsElement e;
-
-    /**
-     * Constructor
-     */
-    public Token(EditsElement e) { this.e = e; }
-
-    /**
-     * EditsElement accessor
-     *
-     * @return EditsElement of this Token
-     */
-    public EditsElement getEditsElement() { return e; }
-
-    /**
-     * Creates token from a string
-     *
-     * @param string a string to set the value of token
-     */
-    abstract public void fromString(String s) throws IOException;
-
-    /**
-     * Creates token from binary stream
-     *
-     * @param in input stream to read token value from
-     */
-    abstract public void fromBinary(DataInputStream in) throws IOException;
-
-    /**
-     * Converts token to string
-     */
-    abstract public String toString();
-
-    /**
-     * Writes token value in binary format to out
-     *
-     * @param out output stream to write value to
-     */
-    abstract public void toBinary(DataOutputStream out) throws IOException;
-  }
-
-  @InterfaceAudience.Private
-  @InterfaceStability.Unstable
-  public class ByteToken extends Token {
-    public byte value;
-    public ByteToken(EditsElement e) { super(e); }
-
-    @Override
-    public void fromString(String s) throws IOException {
-      value = Byte.valueOf(s);
-    }
-
-    @Override
-    public void fromBinary(DataInputStream in) throws IOException {
-      value = in.readByte();
-    }
-    
-    public void fromByte(byte b) {
-      value = b;
-    }
-
-    @Override
-    public String toString() {
-      return Byte.toString(value);
-    }
-
-    @Override
-    public void toBinary(DataOutputStream out) throws IOException {
-      out.writeByte(value);
-    }
-  }
-
-  @InterfaceAudience.Private
-  @InterfaceStability.Unstable
-  public class ShortToken extends Token {
-    public short value;
-    public ShortToken(EditsElement e) { super(e); }
-
-    @Override
-    public void fromString(String s) throws IOException {
-      value = Short.parseShort(s);
-    }
-
-    @Override
-    public void fromBinary(DataInputStream in) throws IOException {
-      value = in.readShort();
-    }
-
-    @Override
-    public String toString() {
-      return Short.toString(value);
-    }
-
-    @Override
-    public void toBinary(DataOutputStream out) throws IOException {
-      out.writeShort(value);
-    }
-  }
-
-  @InterfaceAudience.Private
-  @InterfaceStability.Unstable
-  public class IntToken extends Token {
-    public int value;
-    public IntToken(EditsElement e) { super(e); }
-
-    @Override
-    public void fromString(String s) throws IOException {
-      value = Integer.parseInt(s);
-    }
-
-    @Override
-    public void fromBinary(DataInputStream in) throws IOException {
-      value = in.readInt();
-    }
-
-    @Override
-    public String toString() {
-      return Integer.toString(value);
-    }
-
-    @Override
-    public void toBinary(DataOutputStream out) throws IOException {
-      out.writeInt(value);
-    }
-  }
-
-  @InterfaceAudience.Private
-  @InterfaceStability.Unstable
-  public class VIntToken extends Token {
-    public int value;
-    public VIntToken(EditsElement e) { super(e); }
-
-    @Override
-    public void fromString(String s) throws IOException {
-      value = Integer.parseInt(s);
-    }
-
-    @Override
-    public void fromBinary(DataInputStream in) throws IOException {
-      value = WritableUtils.readVInt(in);
-    }
-
-    @Override
-    public String toString() {
-      return Integer.toString(value);
-    }
-
-    @Override
-    public void toBinary(DataOutputStream out) throws IOException {
-      WritableUtils.writeVInt(out, value);
-    }
-  }
-
-  @InterfaceAudience.Private
-  @InterfaceStability.Unstable
-  public class LongToken extends Token {
-    public long value;
-    public LongToken(EditsElement e) { super(e); }
-
-    @Override
-    public void fromString(String s) throws IOException {
-      value = Long.parseLong(s);
-    }
-
-    @Override
-    public void fromBinary(DataInputStream in) throws IOException {
-      value = in.readLong();
-    }
-
-    @Override
-    public String toString() {
-      return Long.toString(value);
-    }
-
-    @Override
-    public void toBinary(DataOutputStream out) throws IOException {
-      out.writeLong(value);
-    }
-  }
-
-  @InterfaceAudience.Private
-  @InterfaceStability.Unstable
-  public class VLongToken extends Token {
-    public long value;
-    public VLongToken(EditsElement e) { super(e); }
-
-    @Override
-    public void fromString(String s) throws IOException {
-      value = Long.parseLong(s);
-    }
-
-    @Override
-    public void fromBinary(DataInputStream in) throws IOException {
-      value = WritableUtils.readVLong(in);
-    }
-
-    @Override
-    public String toString() {
-      return Long.toString(value);
-    }
-
-    @Override
-    public void toBinary(DataOutputStream out) throws IOException {
-      WritableUtils.writeVLong(out, value);
-    }
-  }
-
-  @InterfaceAudience.Private
-  @InterfaceStability.Unstable
-  public class StringUTF8Token extends Token {
-    public String value;
-    public StringUTF8Token(EditsElement e) { super(e); }
-
-    @Override
-    public void fromString(String s) throws IOException {
-      value = s;
-    }
-
-    @Override
-    public void fromBinary(DataInputStream in) throws IOException {
-      value = DeprecatedUTF8.readString(in);
-    }
-
-    @Override
-    public String toString() {
-      return value;
-    }
-
-    @Override
-    public void toBinary(DataOutputStream out) throws IOException {
-      DeprecatedUTF8.writeString(out, value);
-    }
-  }
-
-  @InterfaceAudience.Private
-  @InterfaceStability.Unstable
-  public class StringTextToken extends Token {
-    public String value;
-    public StringTextToken(EditsElement e) { super(e); }
-
-    @Override
-    public void fromString(String s) throws IOException {
-      value = s;
-    }
-
-    @Override
-    public void fromBinary(DataInputStream in) throws IOException {
-      value = Text.readString(in);
-    }
-
-    @Override
-    public String toString() {
-      return value;
-    }
-
-    @Override
-    public void toBinary(DataOutputStream out) throws IOException {
-      Text.writeString(out, value);
-    }
-  }
-
-  @InterfaceAudience.Private
-  @InterfaceStability.Unstable
-  public class BlobToken extends Token {
-    public byte[] value = null;
-    public BlobToken(EditsElement e, int length) {
-      super(e);
-      value = (length == -1) ? null : new byte[length];
-    }
-
-    @Override
-    public void fromString(String s) throws IOException {
-      value = Base64.decodeBase64(s);
-    }
-
-    @Override
-    public void fromBinary(DataInputStream in) throws IOException {
-      in.readFully(value);
-    }
-
-    @Override
-    public String toString() {
-      return Base64.encodeBase64URLSafeString(value);
-    }
-
-    @Override
-    public void toBinary(DataOutputStream out) throws IOException {
-      out.write(value);
-    }
-  }
-
-  @InterfaceAudience.Private
-  @InterfaceStability.Unstable
-  public class BytesWritableToken extends Token {
-    public BytesWritable value = new BytesWritable();
-    public BytesWritableToken(EditsElement e) { super(e); }
-
-    @Override
-    public void fromString(String s) throws IOException {
-      value = new BytesWritable(Base64.decodeBase64(s));
-    }
-
-    @Override
-    public void fromBinary(DataInputStream in) throws IOException {
-      value.readFields(in);
-    }
-
-    @Override
-    public String toString() {
-      return Base64.encodeBase64URLSafeString(value.getBytes());
-    }
-
-    @Override
-    public void toBinary(DataOutputStream out) throws IOException {
-      value.write(out);
-    }
-  }
-
-  @InterfaceAudience.Private
-  @InterfaceStability.Unstable
-  public class EmptyToken extends Token {
-    public EmptyToken(EditsElement e) { super(e); }
-
-    @Override
-    public void fromString(String s) throws IOException {}
-
-    @Override
-    public void fromBinary(DataInputStream in) throws IOException {}
-
-    @Override
-    public String toString() { return ""; }
-
-    @Override
-    public void toBinary(DataOutputStream out) throws IOException {}
-  }
-
-  /**
-   * Read a Token, note that there is no write function
-   * because writing is handled by Visitor and individual
-   * toString/toBinary functions for given Token implementations.
-   *
-   * Note that it works on the token it gets as a parameter
-   * and returns the same token, this is done so that it can be
-   * called in pipe-like pattern token = f1(f2(f3())), where f3
-   * creates an instance of Token.
-   *
-   * @param t token to read
-   * @return token that was just read
-   */
-  public Token read(Token t) throws IOException;
-}

+ 65 - 113
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/XmlEditsVisitor.java

@@ -18,12 +18,19 @@
 package org.apache.hadoop.hdfs.tools.offlineEditsViewer;
 
 import java.io.IOException;
-import java.util.LinkedList;
+import java.io.OutputStream;
 
-import org.apache.hadoop.hdfs.tools.offlineImageViewer.DepthCounter;
+import org.apache.hadoop.hdfs.util.XMLUtils;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.xml.sax.ContentHandler;
+import org.xml.sax.SAXException;
+import org.xml.sax.helpers.AttributesImpl;
+
+import com.sun.org.apache.xml.internal.serialize.OutputFormat;
+import com.sun.org.apache.xml.internal.serialize.XMLSerializer;
 
 /**
  * An XmlEditsVisitor walks over an EditLog structure and writes out
@@ -31,140 +38,85 @@ import org.apache.hadoop.classification.InterfaceStability;
  */
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
-public class XmlEditsVisitor extends TextEditsVisitor {
-  final private LinkedList<EditsElement> tagQ =
-    new LinkedList<EditsElement>();
-
-  final private DepthCounter depthCounter = new DepthCounter();
-
-  /**
-   * Create a processor that writes to the file named and may or may not
-   * also output to the screen, as specified.
-   *
-   * @param filename Name of file to write output to
-   * @param tokenizer Input tokenizer
-   */
-  public XmlEditsVisitor(String filename, Tokenizer tokenizer)
-    throws IOException {
-
-    super(filename, tokenizer, false);
-  }
+public class XmlEditsVisitor implements OfflineEditsVisitor {
+  private OutputStream out;
+  private ContentHandler contentHandler;
 
   /**
    * Create a processor that writes to the file named and may or may not
    * also output to the screen, as specified.
    *
    * @param filename Name of file to write output to
-   * @param tokenizer Input tokenizer
-   * @param printToScreen Mirror output to screen? (ignored for binary)
+   * @param printToScreen Mirror output to screen?
    */
-  public XmlEditsVisitor(String filename,
-    Tokenizer tokenizer,
-    boolean printToScreen) throws IOException {
-
-    super(filename, tokenizer, printToScreen);
+  public XmlEditsVisitor(OutputStream out)
+      throws IOException {
+    this.out = out;
+    OutputFormat outFormat = new OutputFormat("XML", "UTF-8", true);
+    outFormat.setIndenting(true);
+    outFormat.setIndent(2);
+    outFormat.setDoctype(null, null);
+    XMLSerializer serializer = new XMLSerializer(out, outFormat);
+    contentHandler = serializer.asContentHandler();
+    try {
+      contentHandler.startDocument();
+      contentHandler.startElement("", "", "EDITS", new AttributesImpl());
+    } catch (SAXException e) {
+      throw new IOException("SAX error: " + e.getMessage());
+    }
   }
 
   /**
    * Start visitor (initialization)
    */
   @Override
-  void start() throws IOException {
-    write("<?xml version=\"1.0\"?>\n");
-  }
-
-  /**
-   * Finish visitor
-   */
-  @Override
-  void finish() throws IOException {
-    super.finish();
-  }
-
-  /**
-   * Finish with error
-   */
-  @Override
-  void finishAbnormally() throws IOException {
-    write("\n<!-- Error processing EditLog file.  Exiting -->\n");
-    super.finishAbnormally();
-  }
-
-  /**
-   * Visit a Token
-   *
-   * @param value a Token to visit
-   */
-  @Override
-  Tokenizer.Token visit(Tokenizer.Token value) throws IOException {
-    writeTag(value.getEditsElement().toString(), value.toString());
-    return value;
+  public void start(int version) throws IOException {
+    try {
+      contentHandler.startElement("", "", "EDITS_VERSION", new AttributesImpl());
+      StringBuilder bld = new StringBuilder();
+      bld.append(version);
+      addString(bld.toString());
+      contentHandler.endElement("", "", "EDITS_VERSION");
+    }
+    catch (SAXException e) {
+      throw new IOException("SAX error: " + e.getMessage());
+    }
   }
 
-  /**
-   * Visit an enclosing element (element that cntains other elements)
-   *
-   * @param value a Token to visit
-   */
-  @Override
-  void visitEnclosingElement(Tokenizer.Token value) throws IOException {
-    printIndents();
-    write("<" + value.getEditsElement().toString() + ">\n");
-    tagQ.push(value.getEditsElement());
-    depthCounter.incLevel();
+  public void addString(String str) throws SAXException {
+    int slen = str.length();
+    char arr[] = new char[slen];
+    str.getChars(0, slen, arr, 0);
+    contentHandler.characters(arr, 0, slen);
   }
-
+  
   /**
-   * Leave enclosing element
+   * Finish visitor
    */
   @Override
-  void leaveEnclosingElement() throws IOException {
-    depthCounter.decLevel();
-    if(tagQ.size() == 0)
-      throw new IOException("Tried to exit non-existent enclosing element " +
-                "in EditLog file");
-
-    EditsElement element = tagQ.pop();
-    printIndents();
-    write("</" + element.toString() + ">\n");
-  }
-
-  /**
-   * Write an XML tag
-   *
-   * @param tag a tag name
-   * @param value a tag value
-   */
-  private void writeTag(String tag, String value) throws IOException {
-    printIndents();
-    if(value.length() > 0) {
-      write("<" + tag + ">" + value + "</" + tag + ">\n");
-    } else {
-      write("<" + tag + "/>\n");
+  public void close(Throwable error) throws IOException {
+    try {
+      contentHandler.endElement("", "", "EDITS");
+      if (error != null) {
+        String msg = error.getMessage();
+        XMLUtils.addSaxString(contentHandler, "ERROR",
+            (msg == null) ? "null" : msg);
+      }
+      contentHandler.endDocument();
     }
+    catch (SAXException e) {
+      throw new IOException("SAX error: " + e.getMessage());
+    }
+    out.close();
   }
 
-  // prepared values that printIndents is likely to use
-  final private static String [] indents = {
-     "",
-     "  ",
-     "    ",
-     "      ",
-     "        ",
-     "          ",
-     "            " };
-
-  /**
-   * Prints the leading spaces based on depth level
-   */
-  private void printIndents() throws IOException {
+  @Override
+  public void visitOp(FSEditLogOp op) throws IOException {
     try {
-      write(indents[depthCounter.getLevel()]);
-    } catch (IndexOutOfBoundsException e) {
-      // unlikely needed so can be slow
-      for(int i = 0; i < depthCounter.getLevel(); i++)
-        write("  ");
+      op.outputToXml(contentHandler);
+    }
+    catch (SAXException e) {
+      throw new IOException("SAX error: " + e.getMessage());
     }
-   
   }
 }

+ 0 - 140
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/XmlTokenizer.java

@@ -1,140 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.tools.offlineEditsViewer;
-
-import java.io.IOException;
-import java.io.FileNotFoundException;
-import java.io.FileInputStream;
-
-import javax.xml.stream.XMLStreamException;
-import javax.xml.stream.XMLStreamConstants;
-import javax.xml.stream.XMLInputFactory;
-import javax.xml.stream.XMLStreamReader;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * Tokenizer that reads tokens from XML file
- *
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class XmlTokenizer implements Tokenizer {
-
-  FileInputStream is = null;
-  XMLStreamReader in;
-
-  /**
-   * XmlTokenizer constructor
-   *
-   * @param filename input filename
-   */
-  public XmlTokenizer(String filename) throws IOException {
-    XMLInputFactory f = XMLInputFactory.newInstance();
-    // FileInputStream is = null;
-    try {
-      is = new FileInputStream(filename);
-      in = f.createXMLStreamReader(is);
-    } catch(XMLStreamException e) {
-      // if(is != null) { is.close(); }
-      throw new IOException("Cannot create XML stream", e);
-    } catch(FileNotFoundException e) {
-      //if(is != null) { is.close(); }
-      throw new IOException("Cannot open input file " + filename, e);
-    }
-  }
-
-  /**
-   * Get next element's value, checks that the element's name
-   * is wantedName.
-   *
-   * @param wantedName a name of node that we are looking for
-   */
-  private String getNextElementsValue(String wantedName) throws IOException {
-    boolean gotSTART_ELEMENT = false;
-    try {
-      int eventType = in.getEventType();
-      while(true) {
-        switch(eventType) {
-          case XMLStreamConstants.CHARACTERS: // 4
-            if(gotSTART_ELEMENT) {
-              // XML returns "\n" instead of empty (zero-length) string
-              // for elements like <x></x>
-              return in.getText().trim();
-            }
-            break;
-          case XMLStreamConstants.END_DOCUMENT: // 8
-            throw new IOException("End of XML while looking for element [" +
-              wantedName + "]");
-            // break;
-          case XMLStreamConstants.START_ELEMENT : // 1
-            if(gotSTART_ELEMENT) {
-              throw new IOException("START_ELEMENT [" +
-                in.getName() +
-                " event when expecting CHARACTERS event for [" +
-                wantedName + "]");
-            } else if(in.getName().toString().equals(wantedName)) {
-              gotSTART_ELEMENT = true;
-            } else {
-              throw new IOException("unexpected element name [" +
-                in.getName() + "], was expecting [" +
-                wantedName + "]");
-            }
-            break;
-          case XMLStreamConstants.COMMENT:
-          case XMLStreamConstants.END_ELEMENT: // 2
-          case XMLStreamConstants.SPACE:
-          case XMLStreamConstants.START_DOCUMENT: // 7
-            // these are in XML but we don't need them
-            break;
-          // these should never appear in edits XML
-          case XMLStreamConstants.ATTRIBUTE:
-          case XMLStreamConstants.CDATA:
-          case XMLStreamConstants.DTD:
-          case XMLStreamConstants.ENTITY_DECLARATION:
-          case XMLStreamConstants.ENTITY_REFERENCE:
-          case XMLStreamConstants.NAMESPACE:
-          case XMLStreamConstants.NOTATION_DECLARATION:
-          case XMLStreamConstants.PROCESSING_INSTRUCTION:
-          default:
-            throw new IOException("Unsupported event type [" +
-              eventType + "] (see XMLStreamConstants)");
-        }
-        if(!in.hasNext()) { break; }
-        eventType = in.next();
-      }
-    } catch(XMLStreamException e) {
-      throw new IOException("Error reading XML stream", e);
-    }
-    throw new IOException(
-      "Error reading XML stream, should never reach this line, " +
-      "most likely XML does not have elements we are loking for");
-  }
-
-  /**
-   * @see org.apache.hadoop.hdfs.tools.offlineEditsViewer.Tokenizer#read
-   *
-   * @param t a token to read
-   * @return token that was just read
-   */
-  public Token read(Token t) throws IOException {
-    t.fromString(getNextElementsValue(t.getEditsElement().toString()));
-    return t;
-  }
-}

+ 171 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/XMLUtils.java

@@ -0,0 +1,171 @@
+/**
+ * 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.util;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.xml.sax.ContentHandler;
+import org.xml.sax.SAXException;
+import org.xml.sax.helpers.AttributesImpl;
+
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+/**
+ * General xml utilities.
+ *   
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class XMLUtils {
+  /**
+   * Exception that reflects an invalid XML document.
+   */
+  static public class InvalidXmlException extends RuntimeException {
+    private static final long serialVersionUID = 1L;
+    public InvalidXmlException(String s) {
+      super(s);
+    }
+  }
+  
+  /**
+   * Add a SAX tag with a string inside.
+   *
+   * @param contentHandler     the SAX content handler
+   * @param tag                the element tag to use  
+   * @param value              the string to put inside the tag
+   */
+  public static void addSaxString(ContentHandler contentHandler,
+      String tag, String val) throws SAXException {
+    contentHandler.startElement("", "", tag, new AttributesImpl());
+    char c[] = val.toString().toCharArray();
+    contentHandler.characters(c, 0, c.length);
+    contentHandler.endElement("", "", tag);
+  }
+
+  /**
+   * Represents a bag of key-value pairs encountered during parsing an XML
+   * file.
+   */
+  static public class Stanza {
+    private TreeMap<String, LinkedList <Stanza > > subtrees;
+    private String value;
+    
+    public Stanza() {
+      subtrees = new TreeMap<String, LinkedList <Stanza > >();
+      value = "";
+    }
+    
+    public void setValue(String value) {
+      this.value = value;
+    }
+    
+    public String getValue() {
+      return this.value;
+    }
+    
+    /** 
+     * Discover if a stanza has a given entry.
+     *
+     * @param name        entry to look for
+     * 
+     * @return            true if the entry was found
+     */
+    public boolean hasChildren(String name) {
+      return subtrees.containsKey(name);
+    }
+    
+    /** 
+     * Pull an entry from a stanza.
+     *
+     * @param name        entry to look for
+     * 
+     * @return            the entry
+     */
+    public List<Stanza> getChildren(String name) throws InvalidXmlException {
+      LinkedList <Stanza> children = subtrees.get(name);
+      if (children == null) {
+        throw new InvalidXmlException("no entry found for " + name);
+      }
+      return children;
+    }
+    
+    /** 
+     * Pull a string entry from a stanza.
+     *
+     * @param name        entry to look for
+     * 
+     * @return            the entry
+     */
+    public String getValue(String name) throws InvalidXmlException {
+      if (!subtrees.containsKey(name)) {
+        throw new InvalidXmlException("no entry found for " + name);
+      }
+      LinkedList <Stanza> l = subtrees.get(name);
+      if (l.size() != 1) {
+        throw new InvalidXmlException("More than one value found for " + name);
+      }
+      return l.get(0).getValue();
+    }
+    
+    /** 
+     * Add an entry to a stanza.
+     *
+     * @param name        name of the entry to add
+     * @param child       the entry to add
+     */
+    public void addChild(String name, Stanza child) {
+      LinkedList<Stanza> l;
+      if (subtrees.containsKey(name)) {
+        l = subtrees.get(name);
+      } else {
+        l = new LinkedList<Stanza>();
+        subtrees.put(name, l);
+      }
+      l.add(child);
+    }
+    
+    /** 
+     * Convert a stanza to a human-readable string.
+     */
+    public String toString() {
+      StringBuilder bld = new StringBuilder();
+      bld.append("{");
+      if (!value.equals("")) {
+        bld.append("\"").append(value).append("\"");
+      }
+      String prefix = "";
+      for (Map.Entry<String, LinkedList <Stanza > > entry :
+          subtrees.entrySet()) {
+        String key = entry.getKey();
+        LinkedList <Stanza > ll = entry.getValue();
+        for (Stanza child : ll) {
+          bld.append(prefix);
+          bld.append("<").append(key).append(">");
+          bld.append(child.toString());
+          prefix = ", ";
+        }
+      }
+      bld.append("}");
+      return bld.toString();
+    }
+  }
+}

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

@@ -305,7 +305,7 @@ public class JsonUtil {
     }
 
     return new DatanodeInfo(
-        (String)m.get("name"),
+        (String)m.get("ipAddr"),
         (String)m.get("hostName"),
         (String)m.get("storageID"),
         (int)(long)(Long)m.get("xferPort"),

+ 18 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/JournalProtocol.proto

@@ -27,16 +27,25 @@ option java_generate_equals_and_hash = true;
 import "hdfs.proto";
 
 /**
- * registration - the registration info of the active NameNode
- * firstTxnId - the first txid in the rolled edit log
+ * Journal information used by the journal receiver to identify a journal.
+ */
+message JournalInfoProto {
+  required string clusterID = 1;     // ID of the cluster
+  optional uint32 layoutVersion = 2; // Layout version
+  optional uint32 namespaceID = 3;    // Namespace ID
+}
+
+/**
+ * JournalInfo - the information about the journal
+ * firstTxnId - the first txid in the journal records
  * numTxns - Number of transactions in editlog
  * records - bytes containing serialized journal records
  */
 message JournalRequestProto {
-  required NamenodeRegistrationProto registration = 1; // Registration info
-  required uint64 firstTxnId = 2; // Transaction ID
-  required uint32 numTxns = 3;    // Transaction ID
-  required bytes records = 4;     // Journal record
+  required JournalInfoProto journalInfo = 1;
+  required uint64 firstTxnId = 2;
+  required uint32 numTxns = 3;
+  required bytes records = 4;
 }
 
 /**
@@ -46,12 +55,12 @@ message JournalResponseProto {
 }
 
 /**
- * registration - the registration info of the active NameNode
+ * JournalInfo - the information about the journal
  * txid - first txid in the new log
  */
 message StartLogSegmentRequestProto {
-  required NamenodeRegistrationProto registration = 1; // Registration info
-  required uint64 txid = 2; // Transaction ID
+  required JournalInfoProto journalInfo = 1;
+  required uint64 txid = 2;
 }
 
 /**

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

@@ -395,9 +395,9 @@ public class TestDFSUtil {
     conf.set(HADOOP_SECURITY_AUTHENTICATION, "kerberos");
     UserGroupInformation.setConfiguration(conf);
     String httpsport = DFSUtil.getInfoServer(null, conf, true);
-    assertEquals("0.0.0.0:50470", httpsport);
+    assertEquals("0.0.0.0:"+DFS_NAMENODE_HTTPS_PORT_DEFAULT, httpsport);
     String httpport = DFSUtil.getInfoServer(null, conf, false);
-    assertEquals("0.0.0.0:50070", httpport);
+    assertEquals("0.0.0.0:"+DFS_NAMENODE_HTTP_PORT_DEFAULT, httpport);
   }
   
   @Test

+ 14 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java

@@ -34,6 +34,7 @@ import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
@@ -516,7 +517,8 @@ public class TestDecommission {
     // Now empty hosts file and ensure the datanode is disallowed
     // from talking to namenode, resulting in it's shutdown.
     ArrayList<String>list = new ArrayList<String>();
-    list.add("invalidhost");
+    final String badHostname = "BOGUSHOST";
+    list.add(badHostname);
     writeConfigFile(hostsFile, list);
     
     for (int j = 0; j < numNameNodes; j++) {
@@ -530,6 +532,17 @@ public class TestDecommission {
         info = client.datanodeReport(DatanodeReportType.LIVE);
       }
       assertEquals("Number of live nodes should be 0", 0, info.length);
+      
+      // Test that non-live and bogus hostnames are considered "dead".
+      // The dead report should have an entry for (1) the DN  that is
+      // now considered dead because it is no longer allowed to connect
+      // and (2) the bogus entry in the hosts file (these entries are
+      // always added last)
+      info = client.datanodeReport(DatanodeReportType.DEAD);
+      assertEquals("There should be 2 dead nodes", 2, info.length);
+      DatanodeID id = cluster.getDataNodes().get(0).getDatanodeId();
+      assertEquals(id.getHostName(), info[0].getHostName());
+      assertEquals(badHostname, info[1].getHostName());
     }
   }
 }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReplaceDatanodeOnFailure.java

@@ -60,7 +60,7 @@ public class TestReplaceDatanodeOnFailure {
     final DatanodeInfo[][] datanodes = new DatanodeInfo[infos.length + 1][];
     datanodes[0] = new DatanodeInfo[0];
     for(int i = 0; i < infos.length; ) {
-      infos[i] = new DatanodeInfo(new DatanodeID("dn" + i));
+      infos[i] = new DatanodeInfo(new DatanodeID("dn" + i, 100));
       i++;
       datanodes[i] = new DatanodeInfo[i];
       System.arraycopy(infos, 0, datanodes[i], 0, datanodes[i].length);

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

@@ -115,7 +115,7 @@ public class TestBPOfferService {
             0, HdfsConstants.LAYOUT_VERSION))
       .when(mock).versionRequest();
     
-    Mockito.doReturn(new DatanodeRegistration("fake-node"))
+    Mockito.doReturn(new DatanodeRegistration("fake-node", 100))
       .when(mock).registerDatanode(Mockito.any(DatanodeRegistration.class));
     
     Mockito.doAnswer(new HeartbeatAnswer(nnIdx))

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

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs.server.namenode;
+package org.apache.hadoop.hdfs.server.journalservice;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
@@ -26,7 +26,6 @@ import org.apache.hadoop.fs.FileSystemTestHelper;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.server.namenode.JournalService.JournalListener;
 import org.junit.Test;
 import org.mockito.Mockito;
 

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

@@ -779,9 +779,9 @@ public class NNThroughputBenchmark {
     }
 
     TinyDatanode(int dnIdx, int blockCapacity) throws IOException {
+      String ipAddr = DNS.getDefaultIP("default");
       String hostName = DNS.getDefaultHost("default", "default");
-      dnRegistration = new DatanodeRegistration(hostName);
-      dnRegistration.setXferPort(getNodePort(dnIdx));
+      dnRegistration = new DatanodeRegistration(ipAddr, getNodePort(dnIdx));
       dnRegistration.setHostName(hostName);
       this.blocks = new ArrayList<Block>(blockCapacity);
       this.nrBlocks = 0;
@@ -894,10 +894,10 @@ public class NNThroughputBenchmark {
         for(int t = 0; t < blockTargets.length; t++) {
           DatanodeInfo dnInfo = blockTargets[t];
           DatanodeRegistration receivedDNReg;
-          receivedDNReg = new DatanodeRegistration(dnInfo.getIpAddr());
+          receivedDNReg =
+            new DatanodeRegistration(dnInfo.getIpAddr(), dnInfo.getXferPort());
           receivedDNReg.setStorageInfo(
-                          new DataStorage(nsInfo, dnInfo.getStorageID()));
-          receivedDNReg.setXferPort(dnInfo.getXferPort());
+            new DataStorage(nsInfo, dnInfo.getStorageID()));
           receivedDNReg.setInfoPort(dnInfo.getInfoPort());
           receivedDNReg.setIpcPort(dnInfo.getIpcPort());
           ReceivedDeletedBlockInfo[] rdBlocks = {

+ 22 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestValidateConfigurationSettings.java

@@ -28,6 +28,8 @@ import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.test.GenericTestUtils;
 
 /**
  * This class tests the validation of the configuration object when passed 
@@ -72,6 +74,7 @@ public class TestValidateConfigurationSettings {
     conf.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "127.0.0.1:9000");
     DFSTestUtil.formatNameNode(conf);
     NameNode nameNode = new NameNode(conf); // should be OK!
+    nameNode.stop();
   }
 
   /**
@@ -82,16 +85,30 @@ public class TestValidateConfigurationSettings {
   public void testGenericKeysForNameNodeFormat()
       throws IOException {
     Configuration conf = new HdfsConfiguration();
-    FileSystem.setDefaultUri(conf, "hdfs://localhost:8070");
+
+    // Set ephemeral ports 
+    conf.set(DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY,
+        "127.0.0.1:0");
+    conf.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY,
+        "127.0.0.1:0");
+    
     conf.set(DFSConfigKeys.DFS_FEDERATION_NAMESERVICES, "ns1");
-    String nameDir = System.getProperty("java.io.tmpdir") + "/test.dfs.name";
-    File dir = new File(nameDir);
+    
+    // Set a nameservice-specific configuration for name dir
+    File dir = new File(MiniDFSCluster.getBaseDirectory(),
+        "testGenericKeysForNameNodeFormat");
     if (dir.exists()) {
       FileUtil.fullyDelete(dir);
     }
-    conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY + ".ns1", nameDir);
+    conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY + ".ns1",
+        dir.getAbsolutePath());
+    
+    // Format and verify the right dir is formatted.
     DFSTestUtil.formatNameNode(conf);
+    GenericTestUtils.assertExists(dir);
+
+    // Ensure that the same dir is picked up by the running NN
     NameNode nameNode = new NameNode(conf);
-    FileUtil.fullyDelete(dir);
+    nameNode.stop();
   }
 }

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

@@ -36,7 +36,6 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
-import org.codehaus.jackson.sym.NameN;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;

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

@@ -29,6 +29,7 @@ import org.apache.hadoop.ha.NodeFencer;
 import org.apache.hadoop.ha.ZKFailoverController;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.ha.TestNodeFencer.AlwaysSucceedFencer;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
@@ -56,7 +57,7 @@ public class TestDFSZKFailoverController extends ClientBaseWithFixes {
     // Specify the quorum per-nameservice, to ensure that these configs
     // can be nameservice-scoped.
     conf.set(ZKFailoverController.ZK_QUORUM_KEY + ".ns1", hostPort);
-    conf.set(NodeFencer.CONF_METHODS_KEY,
+    conf.set(DFSConfigKeys.DFS_HA_FENCE_METHODS_KEY,
         AlwaysSucceedFencer.class.getName());
 
     MiniDFSNNTopology topology = new MiniDFSNNTopology()

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

@@ -0,0 +1,120 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode.ha;
+
+import java.io.File;
+import java.io.IOException;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+public class TestInitializeSharedEdits {
+
+  private static final Log LOG = LogFactory.getLog(TestInitializeSharedEdits.class);
+  
+  private static final Path TEST_PATH = new Path("/test");
+  private Configuration conf;
+  private MiniDFSCluster cluster;
+  
+  @Before
+  public void setupCluster() throws IOException {
+    conf = new Configuration();
+
+    MiniDFSNNTopology topology = MiniDFSNNTopology.simpleHATopology();
+    
+    cluster = new MiniDFSCluster.Builder(conf)
+      .nnTopology(topology)
+      .numDataNodes(0)
+      .build();
+    cluster.waitActive();
+  
+    cluster.shutdownNameNode(0);
+    cluster.shutdownNameNode(1);
+    File sharedEditsDir = new File(cluster.getSharedEditsDir(0, 1));
+    assertTrue(FileUtil.fullyDelete(sharedEditsDir));
+  }
+  
+  @After
+  public void shutdownCluster() throws IOException {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+  
+  @Test
+  public void testInitializeSharedEdits() throws Exception {
+    // Make sure we can't currently start either NN.
+    try {
+      cluster.restartNameNode(0, false);
+      fail("Should not have been able to start NN1 without shared dir");
+    } catch (IOException ioe) {
+      LOG.info("Got expected exception", ioe);
+      GenericTestUtils.assertExceptionContains(
+          "Cannot start an HA namenode with name dirs that need recovery", ioe);
+    }
+    try {
+      cluster.restartNameNode(1, false);
+      fail("Should not have been able to start NN2 without shared dir");
+    } catch (IOException ioe) {
+      LOG.info("Got expected exception", ioe);
+      GenericTestUtils.assertExceptionContains(
+          "Cannot start an HA namenode with name dirs that need recovery", ioe);
+    }
+    
+    // Initialize the shared edits dir.
+    assertFalse(NameNode.initializeSharedEdits(conf));
+    
+    // Now should be able to start both NNs. Pass "false" here so that we don't
+    // try to waitActive on all NNs, since the second NN doesn't exist yet.
+    cluster.restartNameNode(0, false);
+    cluster.restartNameNode(1, true);
+    
+    // Make sure HA is working.
+    cluster.transitionToActive(0);
+    FileSystem fs = null;
+    try {
+      fs = HATestUtil.configureFailoverFs(cluster, conf);
+      assertTrue(fs.mkdirs(TEST_PATH));
+      cluster.transitionToStandby(0);
+      cluster.transitionToActive(1);
+      assertTrue(fs.isDirectory(TEST_PATH));
+    } finally {
+      if (fs != null) {
+        fs.close();
+      }
+    }
+  }
+  
+  @Test
+  public void testDontOverWriteExistingDir() {
+    assertFalse(NameNode.initializeSharedEdits(conf, false));
+    assertTrue(NameNode.initializeSharedEdits(conf, false));
+  }
+}

+ 41 - 7
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdmin.java

@@ -158,7 +158,7 @@ public class TestDFSHAAdmin {
   public void testFailoverWithFencerConfigured() throws Exception {
     Mockito.doReturn(STANDBY_READY_RESULT).when(mockProtocol).getServiceStatus();
     HdfsConfiguration conf = getHAConf();
-    conf.set(NodeFencer.CONF_METHODS_KEY, "shell(true)");
+    conf.set(DFSConfigKeys.DFS_HA_FENCE_METHODS_KEY, "shell(true)");
     tool.setConf(conf);
     assertEquals(0, runTool("-failover", "nn1", "nn2"));
   }
@@ -167,7 +167,7 @@ public class TestDFSHAAdmin {
   public void testFailoverWithFencerAndNameservice() throws Exception {
     Mockito.doReturn(STANDBY_READY_RESULT).when(mockProtocol).getServiceStatus();
     HdfsConfiguration conf = getHAConf();
-    conf.set(NodeFencer.CONF_METHODS_KEY, "shell(true)");
+    conf.set(DFSConfigKeys.DFS_HA_FENCE_METHODS_KEY, "shell(true)");
     tool.setConf(conf);
     assertEquals(0, runTool("-ns", "ns1", "-failover", "nn1", "nn2"));
   }
@@ -176,7 +176,7 @@ public class TestDFSHAAdmin {
   public void testFailoverWithFencerConfiguredAndForce() throws Exception {
     Mockito.doReturn(STANDBY_READY_RESULT).when(mockProtocol).getServiceStatus();
     HdfsConfiguration conf = getHAConf();
-    conf.set(NodeFencer.CONF_METHODS_KEY, "shell(true)");
+    conf.set(DFSConfigKeys.DFS_HA_FENCE_METHODS_KEY, "shell(true)");
     tool.setConf(conf);
     assertEquals(0, runTool("-failover", "nn1", "nn2", "--forcefence"));
   }
@@ -185,7 +185,7 @@ public class TestDFSHAAdmin {
   public void testFailoverWithForceActive() throws Exception {
     Mockito.doReturn(STANDBY_READY_RESULT).when(mockProtocol).getServiceStatus();
     HdfsConfiguration conf = getHAConf();
-    conf.set(NodeFencer.CONF_METHODS_KEY, "shell(true)");
+    conf.set(DFSConfigKeys.DFS_HA_FENCE_METHODS_KEY, "shell(true)");
     tool.setConf(conf);
     assertEquals(0, runTool("-failover", "nn1", "nn2", "--forceactive"));
   }
@@ -194,7 +194,7 @@ public class TestDFSHAAdmin {
   public void testFailoverWithInvalidFenceArg() throws Exception {
     Mockito.doReturn(STANDBY_READY_RESULT).when(mockProtocol).getServiceStatus();
     HdfsConfiguration conf = getHAConf();
-    conf.set(NodeFencer.CONF_METHODS_KEY, "shell(true)");
+    conf.set(DFSConfigKeys.DFS_HA_FENCE_METHODS_KEY, "shell(true)");
     tool.setConf(conf);
     assertEquals(-1, runTool("-failover", "nn1", "nn2", "notforcefence"));
   }
@@ -209,7 +209,7 @@ public class TestDFSHAAdmin {
   public void testFailoverWithFenceAndBadFencer() throws Exception {
     Mockito.doReturn(STANDBY_READY_RESULT).when(mockProtocol).getServiceStatus();
     HdfsConfiguration conf = getHAConf();
-    conf.set(NodeFencer.CONF_METHODS_KEY, "foobar!");
+    conf.set(DFSConfigKeys.DFS_HA_FENCE_METHODS_KEY, "foobar!");
     tool.setConf(conf);
     assertEquals(-1, runTool("-failover", "nn1", "nn2", "--forcefence"));
   }
@@ -218,7 +218,7 @@ public class TestDFSHAAdmin {
   public void testForceFenceOptionListedBeforeArgs() throws Exception {
     Mockito.doReturn(STANDBY_READY_RESULT).when(mockProtocol).getServiceStatus();
     HdfsConfiguration conf = getHAConf();
-    conf.set(NodeFencer.CONF_METHODS_KEY, "shell(true)");
+    conf.set(DFSConfigKeys.DFS_HA_FENCE_METHODS_KEY, "shell(true)");
     tool.setConf(conf);
     assertEquals(0, runTool("-failover", "--forcefence", "nn1", "nn2"));
   }
@@ -240,7 +240,41 @@ public class TestDFSHAAdmin {
     assertEquals(-1, runTool("-checkHealth", "nn1"));
     assertOutputContains("Health check failed: fake health check failure");
   }
+  
+  /**
+   * Test that the fencing configuration can be overridden per-nameservice
+   * or per-namenode
+   */
+  @Test
+  public void testFencingConfigPerNameNode() throws Exception {
+    Mockito.doReturn(STANDBY_READY_RESULT).when(mockProtocol).getServiceStatus();
+
+    final String nsSpecificKey = DFSConfigKeys.DFS_HA_FENCE_METHODS_KEY + "." + NSID;
+    final String nnSpecificKey = nsSpecificKey + ".nn1";
+    
+    HdfsConfiguration conf = getHAConf();
+    // Set the default fencer to succeed
+    conf.set(DFSConfigKeys.DFS_HA_FENCE_METHODS_KEY, "shell(true)");
+    tool.setConf(conf);
+    assertEquals(0, runTool("-failover", "nn1", "nn2", "--forcefence"));
+    
+    // Set the NN-specific fencer to fail. Should fail to fence.
+    conf.set(nnSpecificKey, "shell(false)");
+    tool.setConf(conf);
+    assertEquals(-1, runTool("-failover", "nn1", "nn2", "--forcefence"));
+    conf.unset(nnSpecificKey);
 
+    // Set an NS-specific fencer to fail. Should fail.
+    conf.set(nsSpecificKey, "shell(false)");
+    tool.setConf(conf);
+    assertEquals(-1, runTool("-failover", "nn1", "nn2", "--forcefence"));
+    
+    // Set the NS-specific fencer to succeed. Should succeed
+    conf.set(nsSpecificKey, "shell(true)");
+    tool.setConf(conf);
+    assertEquals(0, runTool("-failover", "nn1", "nn2", "--forcefence"));
+  }
+  
   private Object runTool(String ... args) throws Exception {
     errOutBytes.reset();
     LOG.info("Running: DFSHAAdmin " + Joiner.on(" ").join(args));

+ 6 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdminMiniCluster.java

@@ -28,6 +28,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
@@ -113,7 +114,7 @@ public class TestDFSHAAdminMiniCluster {
   
   @Test
   public void testTryFailoverToSafeMode() throws Exception {
-    conf.set(NodeFencer.CONF_METHODS_KEY, "shell(true)");
+    conf.set(DFSConfigKeys.DFS_HA_FENCE_METHODS_KEY, "shell(true)");
     tool.setConf(conf);
 
     NameNodeAdapter.enterSafeMode(cluster.getNameNode(0), false);
@@ -135,7 +136,7 @@ public class TestDFSHAAdminMiniCluster {
     // tmp file, so we can verify that the args were substituted right
     File tmpFile = File.createTempFile("testFencer", ".txt");
     tmpFile.deleteOnExit();
-    conf.set(NodeFencer.CONF_METHODS_KEY,
+    conf.set(DFSConfigKeys.DFS_HA_FENCE_METHODS_KEY,
         "shell(echo -n $target_nameserviceid.$target_namenodeid " +
         "$target_port $dfs_ha_namenode_id > " +
         tmpFile.getAbsolutePath() + ")");
@@ -168,19 +169,19 @@ public class TestDFSHAAdminMiniCluster {
 
           
     // Test failover with not fencer and forcefence option
-    conf.unset(NodeFencer.CONF_METHODS_KEY);
+    conf.unset(DFSConfigKeys.DFS_HA_FENCE_METHODS_KEY);
     tool.setConf(conf);
     assertEquals(-1, runTool("-failover", "nn1", "nn2", "--forcefence"));
     assertFalse(tmpFile.exists());
 
     // Test failover with bad fencer and forcefence option
-    conf.set(NodeFencer.CONF_METHODS_KEY, "foobar!");
+    conf.set(DFSConfigKeys.DFS_HA_FENCE_METHODS_KEY, "foobar!");
     tool.setConf(conf);
     assertEquals(-1, runTool("-failover", "nn1", "nn2", "--forcefence"));
     assertFalse(tmpFile.exists());
 
     // Test failover with force fence listed before the other arguments
-    conf.set(NodeFencer.CONF_METHODS_KEY, "shell(true)");
+    conf.set(DFSConfigKeys.DFS_HA_FENCE_METHODS_KEY, "shell(true)");
     tool.setConf(conf);
     assertEquals(0, runTool("-failover", "--forcefence", "nn1", "nn2"));
   }

+ 9 - 14
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TestOfflineEditsViewer.java

@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hdfs.tools.offlineEditsViewer;
 
+import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.File;
 import java.nio.ByteBuffer;
@@ -33,8 +34,6 @@ import org.apache.commons.logging.LogFactory;
 
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes;
 import org.apache.hadoop.hdfs.tools.offlineEditsViewer.OfflineEditsViewer;
-import org.apache.hadoop.hdfs.tools.offlineEditsViewer.TokenizerFactory;
-import org.apache.hadoop.hdfs.tools.offlineEditsViewer.EditsVisitorFactory;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 
 import org.apache.hadoop.hdfs.server.namenode.OfflineEditsViewerHelper;
@@ -158,11 +157,8 @@ public class TestOfflineEditsViewer {
     LOG.info("Running oev [" + inFilename + "] [" + outFilename + "]");
 
     OfflineEditsViewer oev = new OfflineEditsViewer();
-    oev.go( EditsVisitorFactory.getEditsVisitor(
-      outFilename,
-      processor,
-      TokenizerFactory.getTokenizer(inFilename),
-      false));
+    if (oev.go(inFilename, outFilename, processor, true, false, null) != 0)
+      throw new RuntimeException("oev failed");
   }
 
   /**
@@ -173,14 +169,11 @@ public class TestOfflineEditsViewer {
    */
   private boolean hasAllOpCodes(String inFilename) throws IOException {
     String outFilename = inFilename + ".stats";
-    StatisticsEditsVisitor visitor =
-      (StatisticsEditsVisitor)EditsVisitorFactory.getEditsVisitor(
-        outFilename,
-        "stats",
-        TokenizerFactory.getTokenizer(inFilename),
-        false);
+    FileOutputStream fout = new FileOutputStream(outFilename);
+    StatisticsEditsVisitor visitor = new StatisticsEditsVisitor(fout);
     OfflineEditsViewer oev = new OfflineEditsViewer();
-    oev.go(visitor);
+    if (oev.go(inFilename, outFilename, "stats", false, false, visitor) != 0)
+      return false;
     LOG.info("Statistics for " + inFilename + "\n" +
       visitor.getStatisticsString());
     
@@ -190,6 +183,8 @@ public class TestOfflineEditsViewer {
       if(obsoleteOpCodes.containsKey(opCode)) {
         continue;
       }
+      if (opCode == FSEditLogOpCodes.OP_INVALID)
+        continue;
       Long count = visitor.getStatistics().get(opCode);
       if((count == null) || (count == 0)) {
         hasAllOpCodes = false;

+ 14 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsFileSystemContract.java

@@ -24,6 +24,7 @@ import java.io.IOException;
 import java.io.InputStreamReader;
 import java.net.HttpURLConnection;
 import java.net.URL;
+import java.util.Arrays;
 import java.util.Map;
 
 import javax.servlet.http.HttpServletResponse;
@@ -133,8 +134,20 @@ public class TestWebHdfsFileSystemContract extends FileSystemContractBaseTest {
     final BlockLocation[] expected = cluster.getFileSystem().getFileBlockLocations(
         new Path(f), 0L, 1L);
     assertEquals(expected.length, computed.length);
-    for(int i = 0; i < computed.length; i++) {
+    for (int i = 0; i < computed.length; i++) {
       assertEquals(expected[i].toString(), computed[i].toString());
+      // Check names
+      String names1[] = expected[i].getNames();
+      String names2[] = computed[i].getNames();
+      Arrays.sort(names1);
+      Arrays.sort(names2);
+      Assert.assertArrayEquals("Names differ", names1, names2);
+      // Check topology
+      String topos1[] = expected[i].getTopologyPaths();
+      String topos2[] = computed[i].getTopologyPaths();
+      Arrays.sort(topos1);
+      Arrays.sort(topos2);
+      Assert.assertArrayEquals("Topology differs", topos1, topos2);
     }
   }
 

+ 315 - 364
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml

@@ -1,836 +1,787 @@
-<?xml version="1.0"?>
+<?xml version="1.0" encoding="UTF-8"?>
 <EDITS>
   <EDITS_VERSION>-40</EDITS_VERSION>
   <RECORD>
-    <OPCODE>24</OPCODE>
+    <OPCODE>OP_START_LOG_SEGMENT</OPCODE>
     <DATA>
-      <TRANSACTION_ID>1</TRANSACTION_ID>
+      <TXID>1</TXID>
     </DATA>
-    <CHECKSUM>-2045328303</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>21</OPCODE>
+    <OPCODE>OP_UPDATE_MASTER_KEY</OPCODE>
     <DATA>
-      <TRANSACTION_ID>2</TRANSACTION_ID>
-      <KEY_ID>1</KEY_ID>
-      <KEY_EXPIRY_DATE>1331096884634</KEY_EXPIRY_DATE>
-      <KEY_LENGTH>3</KEY_LENGTH>
-      <KEY_BLOB>o0v1</KEY_BLOB>
+      <TXID>2</TXID>
+      <DELEGATION_KEY>
+        <KEY_ID>1</KEY_ID>
+        <EXPIRY_DATE>1331096884634</EXPIRY_DATE>
+        <KEY>a34bf5</KEY>
+      </DELEGATION_KEY>
     </DATA>
-    <CHECKSUM>-1521490291</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>21</OPCODE>
+    <OPCODE>OP_UPDATE_MASTER_KEY</OPCODE>
     <DATA>
-      <TRANSACTION_ID>3</TRANSACTION_ID>
-      <KEY_ID>2</KEY_ID>
-      <KEY_EXPIRY_DATE>1331096884637</KEY_EXPIRY_DATE>
-      <KEY_LENGTH>3</KEY_LENGTH>
-      <KEY_BLOB>3WMF</KEY_BLOB>
+      <TXID>3</TXID>
+      <DELEGATION_KEY>
+        <KEY_ID>2</KEY_ID>
+        <EXPIRY_DATE>1331096884637</EXPIRY_DATE>
+        <KEY>dd6305</KEY>
+      </DELEGATION_KEY>
     </DATA>
-    <CHECKSUM>65546244</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>10</OPCODE>
+    <OPCODE>OP_SET_GENSTAMP</OPCODE>
     <DATA>
-      <TRANSACTION_ID>4</TRANSACTION_ID>
-      <GENERATION_STAMP>1001</GENERATION_STAMP>
+      <TXID>4</TXID>
+      <GENSTAMP>1001</GENSTAMP>
     </DATA>
-    <CHECKSUM>1423210231</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>0</OPCODE>
+    <OPCODE>OP_ADD</OPCODE>
     <DATA>
-      <TRANSACTION_ID>5</TRANSACTION_ID>
+      <TXID>5</TXID>
+      <LENGTH>0</LENGTH>
       <PATH>/file_create</PATH>
       <REPLICATION>1</REPLICATION>
       <MTIME>1330405685834</MTIME>
       <ATIME>1330405685834</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <NUMBLOCKS>0</NUMBLOCKS>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-2143415023_1</CLIENT_NAME>
+      <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <PERMISSION_STATUS>
         <USERNAME>todd</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
-        <FS_PERMISSIONS>420</FS_PERMISSIONS>
+        <MODE>420</MODE>
       </PERMISSION_STATUS>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-2143415023_1</CLIENT_NAME>
-      <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
     </DATA>
-    <CHECKSUM>179250704</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>9</OPCODE>
+    <OPCODE>OP_CLOSE</OPCODE>
     <DATA>
-      <TRANSACTION_ID>6</TRANSACTION_ID>
+      <TXID>6</TXID>
+      <LENGTH>0</LENGTH>
       <PATH>/file_create</PATH>
       <REPLICATION>1</REPLICATION>
       <MTIME>1330405685848</MTIME>
       <ATIME>1330405685834</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <NUMBLOCKS>0</NUMBLOCKS>
+      <CLIENT_NAME></CLIENT_NAME>
+      <CLIENT_MACHINE></CLIENT_MACHINE>
       <PERMISSION_STATUS>
         <USERNAME>todd</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
-        <FS_PERMISSIONS>420</FS_PERMISSIONS>
+        <MODE>420</MODE>
       </PERMISSION_STATUS>
     </DATA>
-    <CHECKSUM>-584136658</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>1</OPCODE>
+    <OPCODE>OP_RENAME_OLD</OPCODE>
     <DATA>
-      <TRANSACTION_ID>7</TRANSACTION_ID>
-      <SOURCE>/file_create</SOURCE>
-      <DESTINATION>/file_moved</DESTINATION>
+      <TXID>7</TXID>
+      <LENGTH>0</LENGTH>
+      <SRC>/file_create</SRC>
+      <DST>/file_moved</DST>
       <TIMESTAMP>1330405685852</TIMESTAMP>
     </DATA>
-    <CHECKSUM>-1983534581</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>2</OPCODE>
+    <OPCODE>OP_DELETE</OPCODE>
     <DATA>
-      <TRANSACTION_ID>8</TRANSACTION_ID>
+      <TXID>8</TXID>
+      <LENGTH>0</LENGTH>
       <PATH>/file_moved</PATH>
       <TIMESTAMP>1330405685857</TIMESTAMP>
     </DATA>
-    <CHECKSUM>-97648053</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>3</OPCODE>
+    <OPCODE>OP_MKDIR</OPCODE>
     <DATA>
-      <TRANSACTION_ID>9</TRANSACTION_ID>
+      <TXID>9</TXID>
+      <LENGTH>0</LENGTH>
       <PATH>/directory_mkdir</PATH>
       <TIMESTAMP>1330405685861</TIMESTAMP>
-      <ATIME>1330405685861</ATIME>
       <PERMISSION_STATUS>
         <USERNAME>todd</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
-        <FS_PERMISSIONS>493</FS_PERMISSIONS>
+        <MODE>493</MODE>
       </PERMISSION_STATUS>
     </DATA>
-    <CHECKSUM>-146811985</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>10</OPCODE>
+    <OPCODE>OP_SET_GENSTAMP</OPCODE>
     <DATA>
-      <TRANSACTION_ID>10</TRANSACTION_ID>
-      <GENERATION_STAMP>1002</GENERATION_STAMP>
+      <TXID>10</TXID>
+      <GENSTAMP>1002</GENSTAMP>
     </DATA>
-    <CHECKSUM>85982431</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>0</OPCODE>
+    <OPCODE>OP_ADD</OPCODE>
     <DATA>
-      <TRANSACTION_ID>11</TRANSACTION_ID>
+      <TXID>11</TXID>
+      <LENGTH>0</LENGTH>
       <PATH>/file_create</PATH>
       <REPLICATION>1</REPLICATION>
       <MTIME>1330405685866</MTIME>
       <ATIME>1330405685866</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <NUMBLOCKS>0</NUMBLOCKS>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-2143415023_1</CLIENT_NAME>
+      <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <PERMISSION_STATUS>
         <USERNAME>todd</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
-        <FS_PERMISSIONS>420</FS_PERMISSIONS>
+        <MODE>420</MODE>
       </PERMISSION_STATUS>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-2143415023_1</CLIENT_NAME>
-      <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
     </DATA>
-    <CHECKSUM>806955943</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>9</OPCODE>
+    <OPCODE>OP_CLOSE</OPCODE>
     <DATA>
-      <TRANSACTION_ID>12</TRANSACTION_ID>
+      <TXID>12</TXID>
+      <LENGTH>0</LENGTH>
       <PATH>/file_create</PATH>
       <REPLICATION>1</REPLICATION>
       <MTIME>1330405685868</MTIME>
       <ATIME>1330405685866</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <NUMBLOCKS>0</NUMBLOCKS>
+      <CLIENT_NAME></CLIENT_NAME>
+      <CLIENT_MACHINE></CLIENT_MACHINE>
       <PERMISSION_STATUS>
         <USERNAME>todd</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
-        <FS_PERMISSIONS>420</FS_PERMISSIONS>
+        <MODE>420</MODE>
       </PERMISSION_STATUS>
     </DATA>
-    <CHECKSUM>641893387</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>4</OPCODE>
+    <OPCODE>OP_SET_REPLICATION</OPCODE>
     <DATA>
-      <TRANSACTION_ID>13</TRANSACTION_ID>
+      <TXID>13</TXID>
       <PATH>/file_create</PATH>
       <REPLICATION>1</REPLICATION>
     </DATA>
-    <CHECKSUM>24198146</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>7</OPCODE>
+    <OPCODE>OP_SET_PERMISSIONS</OPCODE>
     <DATA>
-      <TRANSACTION_ID>14</TRANSACTION_ID>
-      <PATH>/file_create</PATH>
-      <FS_PERMISSIONS>511</FS_PERMISSIONS>
+      <TXID>14</TXID>
+      <SRC>/file_create</SRC>
+      <MODE>511</MODE>
     </DATA>
-    <CHECKSUM>605568911</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>8</OPCODE>
+    <OPCODE>OP_SET_OWNER</OPCODE>
     <DATA>
-      <TRANSACTION_ID>15</TRANSACTION_ID>
-      <PATH>/file_create</PATH>
+      <TXID>15</TXID>
+      <SRC>/file_create</SRC>
       <USERNAME>newOwner</USERNAME>
-      <GROUPNAME/>
     </DATA>
-    <CHECKSUM>-1411790340</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>13</OPCODE>
+    <OPCODE>OP_TIMES</OPCODE>
     <DATA>
-      <TRANSACTION_ID>16</TRANSACTION_ID>
+      <TXID>16</TXID>
+      <LENGTH>0</LENGTH>
       <PATH>/file_create</PATH>
       <MTIME>1285195527000</MTIME>
       <ATIME>1285195527000</ATIME>
     </DATA>
-    <CHECKSUM>1853168961</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>14</OPCODE>
+    <OPCODE>OP_SET_QUOTA</OPCODE>
     <DATA>
-      <TRANSACTION_ID>17</TRANSACTION_ID>
-      <PATH>/directory_mkdir</PATH>
-      <NS_QUOTA>1000</NS_QUOTA>
-      <DS_QUOTA>-1</DS_QUOTA>
+      <TXID>17</TXID>
+      <SRC>/directory_mkdir</SRC>
+      <NSQUOTA>1000</NSQUOTA>
+      <DSQUOTA>-1</DSQUOTA>
     </DATA>
-    <CHECKSUM>-1476130374</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>15</OPCODE>
+    <OPCODE>OP_RENAME</OPCODE>
     <DATA>
-      <TRANSACTION_ID>18</TRANSACTION_ID>
-      <SOURCE>/file_create</SOURCE>
-      <DESTINATION>/file_moved</DESTINATION>
+      <TXID>18</TXID>
+      <LENGTH>0</LENGTH>
+      <SRC>/file_create</SRC>
+      <DST>/file_moved</DST>
       <TIMESTAMP>1330405685882</TIMESTAMP>
-      <RENAME_OPTIONS>AA</RENAME_OPTIONS>
+      <OPTIONS>NONE</OPTIONS>
     </DATA>
-    <CHECKSUM>-1235158297</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>10</OPCODE>
+    <OPCODE>OP_SET_GENSTAMP</OPCODE>
     <DATA>
-      <TRANSACTION_ID>19</TRANSACTION_ID>
-      <GENERATION_STAMP>1003</GENERATION_STAMP>
+      <TXID>19</TXID>
+      <GENSTAMP>1003</GENSTAMP>
     </DATA>
-    <CHECKSUM>1920677987</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>0</OPCODE>
+    <OPCODE>OP_ADD</OPCODE>
     <DATA>
-      <TRANSACTION_ID>20</TRANSACTION_ID>
+      <TXID>20</TXID>
+      <LENGTH>0</LENGTH>
       <PATH>/file_concat_target</PATH>
       <REPLICATION>1</REPLICATION>
       <MTIME>1330405685889</MTIME>
       <ATIME>1330405685889</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <NUMBLOCKS>0</NUMBLOCKS>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-2143415023_1</CLIENT_NAME>
+      <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <PERMISSION_STATUS>
         <USERNAME>todd</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
-        <FS_PERMISSIONS>420</FS_PERMISSIONS>
+        <MODE>420</MODE>
       </PERMISSION_STATUS>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-2143415023_1</CLIENT_NAME>
-      <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
     </DATA>
-    <CHECKSUM>-981119572</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>10</OPCODE>
+    <OPCODE>OP_SET_GENSTAMP</OPCODE>
     <DATA>
-      <TRANSACTION_ID>21</TRANSACTION_ID>
-      <GENERATION_STAMP>1004</GENERATION_STAMP>
+      <TXID>21</TXID>
+      <GENSTAMP>1004</GENSTAMP>
     </DATA>
-    <CHECKSUM>-1627007926</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>25</OPCODE>
+    <OPCODE>OP_UPDATE_BLOCKS</OPCODE>
     <DATA>
-      <TRANSACTION_ID>22</TRANSACTION_ID>
+      <TXID>22</TXID>
       <PATH>/file_concat_target</PATH>
-      <NUMBLOCKS>1</NUMBLOCKS>
       <BLOCK>
         <BLOCK_ID>-7144805496741076283</BLOCK_ID>
-        <BLOCK_NUM_BYTES>0</BLOCK_NUM_BYTES>
-        <BLOCK_GENERATION_STAMP>1004</BLOCK_GENERATION_STAMP>
+        <NUM_BYTES>0</NUM_BYTES>
+        <GENERATION_STAMP>1004</GENERATION_STAMP>
       </BLOCK>
     </DATA>
-    <CHECKSUM>-1131701615</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>10</OPCODE>
+    <OPCODE>OP_SET_GENSTAMP</OPCODE>
     <DATA>
-      <TRANSACTION_ID>23</TRANSACTION_ID>
-      <GENERATION_STAMP>1005</GENERATION_STAMP>
+      <TXID>23</TXID>
+      <GENSTAMP>1005</GENSTAMP>
     </DATA>
-    <CHECKSUM>-957035430</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>25</OPCODE>
+    <OPCODE>OP_UPDATE_BLOCKS</OPCODE>
     <DATA>
-      <TRANSACTION_ID>24</TRANSACTION_ID>
+      <TXID>24</TXID>
       <PATH>/file_concat_target</PATH>
-      <NUMBLOCKS>2</NUMBLOCKS>
       <BLOCK>
         <BLOCK_ID>-7144805496741076283</BLOCK_ID>
-        <BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
-        <BLOCK_GENERATION_STAMP>1004</BLOCK_GENERATION_STAMP>
+        <NUM_BYTES>512</NUM_BYTES>
+        <GENERATION_STAMP>1004</GENERATION_STAMP>
       </BLOCK>
       <BLOCK>
         <BLOCK_ID>-4125931756867080767</BLOCK_ID>
-        <BLOCK_DELTA_NUM_BYTES>-512</BLOCK_DELTA_NUM_BYTES>
-        <BLOCK_DELTA_GEN_STAMP>1</BLOCK_DELTA_GEN_STAMP>
+        <NUM_BYTES>0</NUM_BYTES>
+        <GENERATION_STAMP>1005</GENERATION_STAMP>
       </BLOCK>
     </DATA>
-    <CHECKSUM>-932985519</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>10</OPCODE>
+    <OPCODE>OP_SET_GENSTAMP</OPCODE>
     <DATA>
-      <TRANSACTION_ID>25</TRANSACTION_ID>
-      <GENERATION_STAMP>1006</GENERATION_STAMP>
+      <TXID>25</TXID>
+      <GENSTAMP>1006</GENSTAMP>
     </DATA>
-    <CHECKSUM>-1757460878</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>25</OPCODE>
+    <OPCODE>OP_UPDATE_BLOCKS</OPCODE>
     <DATA>
-      <TRANSACTION_ID>26</TRANSACTION_ID>
+      <TXID>26</TXID>
       <PATH>/file_concat_target</PATH>
-      <NUMBLOCKS>3</NUMBLOCKS>
       <BLOCK>
         <BLOCK_ID>-7144805496741076283</BLOCK_ID>
-        <BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
-        <BLOCK_GENERATION_STAMP>1004</BLOCK_GENERATION_STAMP>
+        <NUM_BYTES>512</NUM_BYTES>
+        <GENERATION_STAMP>1004</GENERATION_STAMP>
       </BLOCK>
       <BLOCK>
         <BLOCK_ID>-4125931756867080767</BLOCK_ID>
-        <BLOCK_DELTA_NUM_BYTES>0</BLOCK_DELTA_NUM_BYTES>
-        <BLOCK_DELTA_GEN_STAMP>1</BLOCK_DELTA_GEN_STAMP>
+        <NUM_BYTES>512</NUM_BYTES>
+        <GENERATION_STAMP>1005</GENERATION_STAMP>
       </BLOCK>
       <BLOCK>
         <BLOCK_ID>1562413691487277050</BLOCK_ID>
-        <BLOCK_DELTA_NUM_BYTES>-512</BLOCK_DELTA_NUM_BYTES>
-        <BLOCK_DELTA_GEN_STAMP>1</BLOCK_DELTA_GEN_STAMP>
+        <NUM_BYTES>0</NUM_BYTES>
+        <GENERATION_STAMP>1006</GENERATION_STAMP>
       </BLOCK>
     </DATA>
-    <CHECKSUM>-154090859</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>9</OPCODE>
+    <OPCODE>OP_CLOSE</OPCODE>
     <DATA>
-      <TRANSACTION_ID>27</TRANSACTION_ID>
+      <TXID>27</TXID>
+      <LENGTH>0</LENGTH>
       <PATH>/file_concat_target</PATH>
       <REPLICATION>1</REPLICATION>
       <MTIME>1330405685978</MTIME>
       <ATIME>1330405685889</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <NUMBLOCKS>3</NUMBLOCKS>
+      <CLIENT_NAME></CLIENT_NAME>
+      <CLIENT_MACHINE></CLIENT_MACHINE>
       <BLOCK>
         <BLOCK_ID>-7144805496741076283</BLOCK_ID>
-        <BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
-        <BLOCK_GENERATION_STAMP>1004</BLOCK_GENERATION_STAMP>
+        <NUM_BYTES>512</NUM_BYTES>
+        <GENERATION_STAMP>1004</GENERATION_STAMP>
       </BLOCK>
       <BLOCK>
         <BLOCK_ID>-4125931756867080767</BLOCK_ID>
-        <BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
-        <BLOCK_GENERATION_STAMP>1005</BLOCK_GENERATION_STAMP>
+        <NUM_BYTES>512</NUM_BYTES>
+        <GENERATION_STAMP>1005</GENERATION_STAMP>
       </BLOCK>
       <BLOCK>
         <BLOCK_ID>1562413691487277050</BLOCK_ID>
-        <BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
-        <BLOCK_GENERATION_STAMP>1006</BLOCK_GENERATION_STAMP>
+        <NUM_BYTES>512</NUM_BYTES>
+        <GENERATION_STAMP>1006</GENERATION_STAMP>
       </BLOCK>
       <PERMISSION_STATUS>
         <USERNAME>todd</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
-        <FS_PERMISSIONS>420</FS_PERMISSIONS>
+        <MODE>420</MODE>
       </PERMISSION_STATUS>
     </DATA>
-    <CHECKSUM>-292633850</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>10</OPCODE>
+    <OPCODE>OP_SET_GENSTAMP</OPCODE>
     <DATA>
-      <TRANSACTION_ID>28</TRANSACTION_ID>
-      <GENERATION_STAMP>1007</GENERATION_STAMP>
+      <TXID>28</TXID>
+      <GENSTAMP>1007</GENSTAMP>
     </DATA>
-    <CHECKSUM>-1431358549</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>0</OPCODE>
+    <OPCODE>OP_ADD</OPCODE>
     <DATA>
-      <TRANSACTION_ID>29</TRANSACTION_ID>
+      <TXID>29</TXID>
+      <LENGTH>0</LENGTH>
       <PATH>/file_concat_0</PATH>
       <REPLICATION>1</REPLICATION>
       <MTIME>1330405685983</MTIME>
       <ATIME>1330405685983</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <NUMBLOCKS>0</NUMBLOCKS>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-2143415023_1</CLIENT_NAME>
+      <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <PERMISSION_STATUS>
         <USERNAME>todd</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
-        <FS_PERMISSIONS>420</FS_PERMISSIONS>
+        <MODE>420</MODE>
       </PERMISSION_STATUS>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-2143415023_1</CLIENT_NAME>
-      <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
     </DATA>
-    <CHECKSUM>-318194869</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>10</OPCODE>
+    <OPCODE>OP_SET_GENSTAMP</OPCODE>
     <DATA>
-      <TRANSACTION_ID>30</TRANSACTION_ID>
-      <GENERATION_STAMP>1008</GENERATION_STAMP>
+      <TXID>30</TXID>
+      <GENSTAMP>1008</GENSTAMP>
     </DATA>
-    <CHECKSUM>156309208</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>25</OPCODE>
+    <OPCODE>OP_UPDATE_BLOCKS</OPCODE>
     <DATA>
-      <TRANSACTION_ID>31</TRANSACTION_ID>
+      <TXID>31</TXID>
       <PATH>/file_concat_0</PATH>
-      <NUMBLOCKS>1</NUMBLOCKS>
       <BLOCK>
         <BLOCK_ID>6084289468290363112</BLOCK_ID>
-        <BLOCK_NUM_BYTES>0</BLOCK_NUM_BYTES>
-        <BLOCK_GENERATION_STAMP>1008</BLOCK_GENERATION_STAMP>
+        <NUM_BYTES>0</NUM_BYTES>
+        <GENERATION_STAMP>1008</GENERATION_STAMP>
       </BLOCK>
     </DATA>
-    <CHECKSUM>-596016492</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>10</OPCODE>
+    <OPCODE>OP_SET_GENSTAMP</OPCODE>
     <DATA>
-      <TRANSACTION_ID>32</TRANSACTION_ID>
-      <GENERATION_STAMP>1009</GENERATION_STAMP>
+      <TXID>32</TXID>
+      <GENSTAMP>1009</GENSTAMP>
     </DATA>
-    <CHECKSUM>-1734001394</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>25</OPCODE>
+    <OPCODE>OP_UPDATE_BLOCKS</OPCODE>
     <DATA>
-      <TRANSACTION_ID>33</TRANSACTION_ID>
+      <TXID>33</TXID>
       <PATH>/file_concat_0</PATH>
-      <NUMBLOCKS>2</NUMBLOCKS>
       <BLOCK>
         <BLOCK_ID>6084289468290363112</BLOCK_ID>
-        <BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
-        <BLOCK_GENERATION_STAMP>1008</BLOCK_GENERATION_STAMP>
+        <NUM_BYTES>512</NUM_BYTES>
+        <GENERATION_STAMP>1008</GENERATION_STAMP>
       </BLOCK>
       <BLOCK>
         <BLOCK_ID>-4219431127125026105</BLOCK_ID>
-        <BLOCK_DELTA_NUM_BYTES>-512</BLOCK_DELTA_NUM_BYTES>
-        <BLOCK_DELTA_GEN_STAMP>1</BLOCK_DELTA_GEN_STAMP>
+        <NUM_BYTES>0</NUM_BYTES>
+        <GENERATION_STAMP>1009</GENERATION_STAMP>
       </BLOCK>
     </DATA>
-    <CHECKSUM>1352178323</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>10</OPCODE>
+    <OPCODE>OP_SET_GENSTAMP</OPCODE>
     <DATA>
-      <TRANSACTION_ID>34</TRANSACTION_ID>
-      <GENERATION_STAMP>1010</GENERATION_STAMP>
+      <TXID>34</TXID>
+      <GENSTAMP>1010</GENSTAMP>
     </DATA>
-    <CHECKSUM>794444850</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>25</OPCODE>
+    <OPCODE>OP_UPDATE_BLOCKS</OPCODE>
     <DATA>
-      <TRANSACTION_ID>35</TRANSACTION_ID>
+      <TXID>35</TXID>
       <PATH>/file_concat_0</PATH>
-      <NUMBLOCKS>3</NUMBLOCKS>
       <BLOCK>
         <BLOCK_ID>6084289468290363112</BLOCK_ID>
-        <BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
-        <BLOCK_GENERATION_STAMP>1008</BLOCK_GENERATION_STAMP>
+        <NUM_BYTES>512</NUM_BYTES>
+        <GENERATION_STAMP>1008</GENERATION_STAMP>
       </BLOCK>
       <BLOCK>
         <BLOCK_ID>-4219431127125026105</BLOCK_ID>
-        <BLOCK_DELTA_NUM_BYTES>0</BLOCK_DELTA_NUM_BYTES>
-        <BLOCK_DELTA_GEN_STAMP>1</BLOCK_DELTA_GEN_STAMP>
+        <NUM_BYTES>512</NUM_BYTES>
+        <GENERATION_STAMP>1009</GENERATION_STAMP>
       </BLOCK>
       <BLOCK>
         <BLOCK_ID>-1765119074945211374</BLOCK_ID>
-        <BLOCK_DELTA_NUM_BYTES>-512</BLOCK_DELTA_NUM_BYTES>
-        <BLOCK_DELTA_GEN_STAMP>1</BLOCK_DELTA_GEN_STAMP>
+        <NUM_BYTES>0</NUM_BYTES>
+        <GENERATION_STAMP>1010</GENERATION_STAMP>
       </BLOCK>
     </DATA>
-    <CHECKSUM>-1530696539</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>9</OPCODE>
+    <OPCODE>OP_CLOSE</OPCODE>
     <DATA>
-      <TRANSACTION_ID>36</TRANSACTION_ID>
+      <TXID>36</TXID>
+      <LENGTH>0</LENGTH>
       <PATH>/file_concat_0</PATH>
       <REPLICATION>1</REPLICATION>
       <MTIME>1330405686013</MTIME>
       <ATIME>1330405685983</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <NUMBLOCKS>3</NUMBLOCKS>
+      <CLIENT_NAME></CLIENT_NAME>
+      <CLIENT_MACHINE></CLIENT_MACHINE>
       <BLOCK>
         <BLOCK_ID>6084289468290363112</BLOCK_ID>
-        <BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
-        <BLOCK_GENERATION_STAMP>1008</BLOCK_GENERATION_STAMP>
+        <NUM_BYTES>512</NUM_BYTES>
+        <GENERATION_STAMP>1008</GENERATION_STAMP>
       </BLOCK>
       <BLOCK>
         <BLOCK_ID>-4219431127125026105</BLOCK_ID>
-        <BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
-        <BLOCK_GENERATION_STAMP>1009</BLOCK_GENERATION_STAMP>
+        <NUM_BYTES>512</NUM_BYTES>
+        <GENERATION_STAMP>1009</GENERATION_STAMP>
       </BLOCK>
       <BLOCK>
         <BLOCK_ID>-1765119074945211374</BLOCK_ID>
-        <BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
-        <BLOCK_GENERATION_STAMP>1010</BLOCK_GENERATION_STAMP>
+        <NUM_BYTES>512</NUM_BYTES>
+        <GENERATION_STAMP>1010</GENERATION_STAMP>
       </BLOCK>
       <PERMISSION_STATUS>
         <USERNAME>todd</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
-        <FS_PERMISSIONS>420</FS_PERMISSIONS>
+        <MODE>420</MODE>
       </PERMISSION_STATUS>
     </DATA>
-    <CHECKSUM>-2043978220</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>10</OPCODE>
+    <OPCODE>OP_SET_GENSTAMP</OPCODE>
     <DATA>
-      <TRANSACTION_ID>37</TRANSACTION_ID>
-      <GENERATION_STAMP>1011</GENERATION_STAMP>
+      <TXID>37</TXID>
+      <GENSTAMP>1011</GENSTAMP>
     </DATA>
-    <CHECKSUM>1010571629</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>0</OPCODE>
+    <OPCODE>OP_ADD</OPCODE>
     <DATA>
-      <TRANSACTION_ID>38</TRANSACTION_ID>
+      <TXID>38</TXID>
+      <LENGTH>0</LENGTH>
       <PATH>/file_concat_1</PATH>
       <REPLICATION>1</REPLICATION>
       <MTIME>1330405686017</MTIME>
       <ATIME>1330405686017</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <NUMBLOCKS>0</NUMBLOCKS>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-2143415023_1</CLIENT_NAME>
+      <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <PERMISSION_STATUS>
         <USERNAME>todd</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
-        <FS_PERMISSIONS>420</FS_PERMISSIONS>
+        <MODE>420</MODE>
       </PERMISSION_STATUS>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-2143415023_1</CLIENT_NAME>
-      <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
     </DATA>
-    <CHECKSUM>-501297097</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>10</OPCODE>
+    <OPCODE>OP_SET_GENSTAMP</OPCODE>
     <DATA>
-      <TRANSACTION_ID>39</TRANSACTION_ID>
-      <GENERATION_STAMP>1012</GENERATION_STAMP>
+      <TXID>39</TXID>
+      <GENSTAMP>1012</GENSTAMP>
     </DATA>
-    <CHECKSUM>-1934711736</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>25</OPCODE>
+    <OPCODE>OP_UPDATE_BLOCKS</OPCODE>
     <DATA>
-      <TRANSACTION_ID>40</TRANSACTION_ID>
+      <TXID>40</TXID>
       <PATH>/file_concat_1</PATH>
-      <NUMBLOCKS>1</NUMBLOCKS>
       <BLOCK>
         <BLOCK_ID>-7448471719302683860</BLOCK_ID>
-        <BLOCK_NUM_BYTES>0</BLOCK_NUM_BYTES>
-        <BLOCK_GENERATION_STAMP>1012</BLOCK_GENERATION_STAMP>
+        <NUM_BYTES>0</NUM_BYTES>
+        <GENERATION_STAMP>1012</GENERATION_STAMP>
       </BLOCK>
     </DATA>
-    <CHECKSUM>-1853122907</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>10</OPCODE>
+    <OPCODE>OP_SET_GENSTAMP</OPCODE>
     <DATA>
-      <TRANSACTION_ID>41</TRANSACTION_ID>
-      <GENERATION_STAMP>1013</GENERATION_STAMP>
+      <TXID>41</TXID>
+      <GENSTAMP>1013</GENSTAMP>
     </DATA>
-    <CHECKSUM>862670668</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>25</OPCODE>
+    <OPCODE>OP_UPDATE_BLOCKS</OPCODE>
     <DATA>
-      <TRANSACTION_ID>42</TRANSACTION_ID>
+      <TXID>42</TXID>
       <PATH>/file_concat_1</PATH>
-      <NUMBLOCKS>2</NUMBLOCKS>
       <BLOCK>
         <BLOCK_ID>-7448471719302683860</BLOCK_ID>
-        <BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
-        <BLOCK_GENERATION_STAMP>1012</BLOCK_GENERATION_STAMP>
+        <NUM_BYTES>512</NUM_BYTES>
+        <GENERATION_STAMP>1012</GENERATION_STAMP>
       </BLOCK>
       <BLOCK>
         <BLOCK_ID>-8051065559769974521</BLOCK_ID>
-        <BLOCK_DELTA_NUM_BYTES>-512</BLOCK_DELTA_NUM_BYTES>
-        <BLOCK_DELTA_GEN_STAMP>1</BLOCK_DELTA_GEN_STAMP>
+        <NUM_BYTES>0</NUM_BYTES>
+        <GENERATION_STAMP>1013</GENERATION_STAMP>
       </BLOCK>
     </DATA>
-    <CHECKSUM>-1169706939</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>10</OPCODE>
+    <OPCODE>OP_SET_GENSTAMP</OPCODE>
     <DATA>
-      <TRANSACTION_ID>43</TRANSACTION_ID>
-      <GENERATION_STAMP>1014</GENERATION_STAMP>
+      <TXID>43</TXID>
+      <GENSTAMP>1014</GENSTAMP>
     </DATA>
-    <CHECKSUM>-2070661520</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>25</OPCODE>
+    <OPCODE>OP_UPDATE_BLOCKS</OPCODE>
     <DATA>
-      <TRANSACTION_ID>44</TRANSACTION_ID>
+      <TXID>44</TXID>
       <PATH>/file_concat_1</PATH>
-      <NUMBLOCKS>3</NUMBLOCKS>
       <BLOCK>
         <BLOCK_ID>-7448471719302683860</BLOCK_ID>
-        <BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
-        <BLOCK_GENERATION_STAMP>1012</BLOCK_GENERATION_STAMP>
+        <NUM_BYTES>512</NUM_BYTES>
+        <GENERATION_STAMP>1012</GENERATION_STAMP>
       </BLOCK>
       <BLOCK>
         <BLOCK_ID>-8051065559769974521</BLOCK_ID>
-        <BLOCK_DELTA_NUM_BYTES>0</BLOCK_DELTA_NUM_BYTES>
-        <BLOCK_DELTA_GEN_STAMP>1</BLOCK_DELTA_GEN_STAMP>
+        <NUM_BYTES>512</NUM_BYTES>
+        <GENERATION_STAMP>1013</GENERATION_STAMP>
       </BLOCK>
       <BLOCK>
         <BLOCK_ID>3808670437711973616</BLOCK_ID>
-        <BLOCK_DELTA_NUM_BYTES>-512</BLOCK_DELTA_NUM_BYTES>
-        <BLOCK_DELTA_GEN_STAMP>1</BLOCK_DELTA_GEN_STAMP>
+        <NUM_BYTES>0</NUM_BYTES>
+        <GENERATION_STAMP>1014</GENERATION_STAMP>
       </BLOCK>
     </DATA>
-    <CHECKSUM>-1568093815</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>9</OPCODE>
+    <OPCODE>OP_CLOSE</OPCODE>
     <DATA>
-      <TRANSACTION_ID>45</TRANSACTION_ID>
+      <TXID>45</TXID>
+      <LENGTH>0</LENGTH>
       <PATH>/file_concat_1</PATH>
       <REPLICATION>1</REPLICATION>
       <MTIME>1330405686042</MTIME>
       <ATIME>1330405686017</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <NUMBLOCKS>3</NUMBLOCKS>
+      <CLIENT_NAME></CLIENT_NAME>
+      <CLIENT_MACHINE></CLIENT_MACHINE>
       <BLOCK>
         <BLOCK_ID>-7448471719302683860</BLOCK_ID>
-        <BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
-        <BLOCK_GENERATION_STAMP>1012</BLOCK_GENERATION_STAMP>
+        <NUM_BYTES>512</NUM_BYTES>
+        <GENERATION_STAMP>1012</GENERATION_STAMP>
       </BLOCK>
       <BLOCK>
         <BLOCK_ID>-8051065559769974521</BLOCK_ID>
-        <BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
-        <BLOCK_GENERATION_STAMP>1013</BLOCK_GENERATION_STAMP>
+        <NUM_BYTES>512</NUM_BYTES>
+        <GENERATION_STAMP>1013</GENERATION_STAMP>
       </BLOCK>
       <BLOCK>
         <BLOCK_ID>3808670437711973616</BLOCK_ID>
-        <BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
-        <BLOCK_GENERATION_STAMP>1014</BLOCK_GENERATION_STAMP>
+        <NUM_BYTES>512</NUM_BYTES>
+        <GENERATION_STAMP>1014</GENERATION_STAMP>
       </BLOCK>
       <PERMISSION_STATUS>
         <USERNAME>todd</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
-        <FS_PERMISSIONS>420</FS_PERMISSIONS>
+        <MODE>420</MODE>
       </PERMISSION_STATUS>
     </DATA>
-    <CHECKSUM>-1640101896</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>16</OPCODE>
+    <OPCODE>OP_CONCAT_DELETE</OPCODE>
     <DATA>
-      <TRANSACTION_ID>46</TRANSACTION_ID>
-      <CONCAT_TARGET>/file_concat_target</CONCAT_TARGET>
-      <LENGTH>2</LENGTH>
-      <CONCAT_SOURCE>/file_concat_0</CONCAT_SOURCE>
-      <CONCAT_SOURCE>/file_concat_1</CONCAT_SOURCE>
+      <TXID>46</TXID>
+      <LENGTH>0</LENGTH>
+      <TRG>/file_concat_target</TRG>
       <TIMESTAMP>1330405686046</TIMESTAMP>
+      <SOURCES>
+        <SOURCE1>/file_concat_0</SOURCE1>
+        <SOURCE2>/file_concat_1</SOURCE2>
+      </SOURCES>
     </DATA>
-    <CHECKSUM>2122891157</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>17</OPCODE>
+    <OPCODE>OP_SYMLINK</OPCODE>
     <DATA>
-      <TRANSACTION_ID>47</TRANSACTION_ID>
-      <SOURCE>/file_symlink</SOURCE>
-      <DESTINATION>/file_concat_target</DESTINATION>
+      <TXID>47</TXID>
+      <LENGTH>0</LENGTH>
+      <PATH>/file_symlink</PATH>
+      <VALUE>/file_concat_target</VALUE>
       <MTIME>1330405686051</MTIME>
       <ATIME>1330405686051</ATIME>
       <PERMISSION_STATUS>
         <USERNAME>todd</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
-        <FS_PERMISSIONS>511</FS_PERMISSIONS>
+        <MODE>511</MODE>
       </PERMISSION_STATUS>
     </DATA>
-    <CHECKSUM>-585385283</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>18</OPCODE>
+    <OPCODE>OP_GET_DELEGATION_TOKEN</OPCODE>
     <DATA>
-      <TRANSACTION_ID>48</TRANSACTION_ID>
-      <T_VERSION>0</T_VERSION>
-      <T_OWNER>todd</T_OWNER>
-      <T_RENEWER>JobTracker</T_RENEWER>
-      <T_REAL_USER/>
-      <T_ISSUE_DATE>1330405686056</T_ISSUE_DATE>
-      <T_MAX_DATE>1331010486056</T_MAX_DATE>
-      <T_SEQUENCE_NUMBER>1</T_SEQUENCE_NUMBER>
-      <T_MASTER_KEY_ID>2</T_MASTER_KEY_ID>
-      <T_EXPIRY_TIME>1330492086056</T_EXPIRY_TIME>
+      <TXID>48</TXID>
+      <DELEGATION_TOKEN_IDENTIFIER>
+        <KIND>HDFS_DELEGATION_TOKEN</KIND>
+        <SEQUENCE_NUMBER>1</SEQUENCE_NUMBER>
+        <OWNER>todd</OWNER>
+        <RENEWER>JobTracker</RENEWER>
+        <REALUSER></REALUSER>
+        <ISSUE_DATE>1330405686056</ISSUE_DATE>
+        <MAX_DATE>1331010486056</MAX_DATE>
+        <MASTER_KEY_ID>2</MASTER_KEY_ID>
+      </DELEGATION_TOKEN_IDENTIFIER>
+      <EXPIRY_TIME>1330492086056</EXPIRY_TIME>
     </DATA>
-    <CHECKSUM>791321007</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>19</OPCODE>
+    <OPCODE>OP_RENEW_DELEGATION_TOKEN</OPCODE>
     <DATA>
-      <TRANSACTION_ID>49</TRANSACTION_ID>
-      <T_VERSION>0</T_VERSION>
-      <T_OWNER>todd</T_OWNER>
-      <T_RENEWER>JobTracker</T_RENEWER>
-      <T_REAL_USER/>
-      <T_ISSUE_DATE>1330405686056</T_ISSUE_DATE>
-      <T_MAX_DATE>1331010486056</T_MAX_DATE>
-      <T_SEQUENCE_NUMBER>1</T_SEQUENCE_NUMBER>
-      <T_MASTER_KEY_ID>2</T_MASTER_KEY_ID>
-      <T_EXPIRY_TIME>1330492086075</T_EXPIRY_TIME>
+      <TXID>49</TXID>
+      <DELEGATION_TOKEN_IDENTIFIER>
+        <KIND>HDFS_DELEGATION_TOKEN</KIND>
+        <SEQUENCE_NUMBER>1</SEQUENCE_NUMBER>
+        <OWNER>todd</OWNER>
+        <RENEWER>JobTracker</RENEWER>
+        <REALUSER></REALUSER>
+        <ISSUE_DATE>1330405686056</ISSUE_DATE>
+        <MAX_DATE>1331010486056</MAX_DATE>
+        <MASTER_KEY_ID>2</MASTER_KEY_ID>
+      </DELEGATION_TOKEN_IDENTIFIER>
+      <EXPIRY_TIME>1330492086075</EXPIRY_TIME>
     </DATA>
-    <CHECKSUM>649714969</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>20</OPCODE>
+    <OPCODE>OP_CANCEL_DELEGATION_TOKEN</OPCODE>
     <DATA>
-      <TRANSACTION_ID>50</TRANSACTION_ID>
-      <T_VERSION>0</T_VERSION>
-      <T_OWNER>todd</T_OWNER>
-      <T_RENEWER>JobTracker</T_RENEWER>
-      <T_REAL_USER/>
-      <T_ISSUE_DATE>1330405686056</T_ISSUE_DATE>
-      <T_MAX_DATE>1331010486056</T_MAX_DATE>
-      <T_SEQUENCE_NUMBER>1</T_SEQUENCE_NUMBER>
-      <T_MASTER_KEY_ID>2</T_MASTER_KEY_ID>
+      <TXID>50</TXID>
+      <DELEGATION_TOKEN_IDENTIFIER>
+        <KIND>HDFS_DELEGATION_TOKEN</KIND>
+        <SEQUENCE_NUMBER>1</SEQUENCE_NUMBER>
+        <OWNER>todd</OWNER>
+        <RENEWER>JobTracker</RENEWER>
+        <REALUSER></REALUSER>
+        <ISSUE_DATE>1330405686056</ISSUE_DATE>
+        <MAX_DATE>1331010486056</MAX_DATE>
+        <MASTER_KEY_ID>2</MASTER_KEY_ID>
+      </DELEGATION_TOKEN_IDENTIFIER>
     </DATA>
-    <CHECKSUM>1190872628</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>10</OPCODE>
+    <OPCODE>OP_SET_GENSTAMP</OPCODE>
     <DATA>
-      <TRANSACTION_ID>51</TRANSACTION_ID>
-      <GENERATION_STAMP>1015</GENERATION_STAMP>
+      <TXID>51</TXID>
+      <GENSTAMP>1015</GENSTAMP>
     </DATA>
-    <CHECKSUM>-460593521</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>0</OPCODE>
+    <OPCODE>OP_ADD</OPCODE>
     <DATA>
-      <TRANSACTION_ID>52</TRANSACTION_ID>
+      <TXID>52</TXID>
+      <LENGTH>0</LENGTH>
       <PATH>/hard-lease-recovery-test</PATH>
       <REPLICATION>1</REPLICATION>
       <MTIME>1330405686084</MTIME>
       <ATIME>1330405686084</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <NUMBLOCKS>0</NUMBLOCKS>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-2143415023_1</CLIENT_NAME>
+      <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <PERMISSION_STATUS>
         <USERNAME>todd</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
-        <FS_PERMISSIONS>420</FS_PERMISSIONS>
+        <MODE>420</MODE>
       </PERMISSION_STATUS>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-2143415023_1</CLIENT_NAME>
-      <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
     </DATA>
-    <CHECKSUM>2093219037</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>10</OPCODE>
+    <OPCODE>OP_SET_GENSTAMP</OPCODE>
     <DATA>
-      <TRANSACTION_ID>53</TRANSACTION_ID>
-      <GENERATION_STAMP>1016</GENERATION_STAMP>
+      <TXID>53</TXID>
+      <GENSTAMP>1016</GENSTAMP>
     </DATA>
-    <CHECKSUM>120488596</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>25</OPCODE>
+    <OPCODE>OP_UPDATE_BLOCKS</OPCODE>
     <DATA>
-      <TRANSACTION_ID>54</TRANSACTION_ID>
+      <TXID>54</TXID>
       <PATH>/hard-lease-recovery-test</PATH>
-      <NUMBLOCKS>1</NUMBLOCKS>
       <BLOCK>
         <BLOCK_ID>-357061736603024522</BLOCK_ID>
-        <BLOCK_NUM_BYTES>0</BLOCK_NUM_BYTES>
-        <BLOCK_GENERATION_STAMP>1016</BLOCK_GENERATION_STAMP>
+        <NUM_BYTES>0</NUM_BYTES>
+        <GENERATION_STAMP>1016</GENERATION_STAMP>
       </BLOCK>
     </DATA>
-    <CHECKSUM>2098840974</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>25</OPCODE>
+    <OPCODE>OP_UPDATE_BLOCKS</OPCODE>
     <DATA>
-      <TRANSACTION_ID>55</TRANSACTION_ID>
+      <TXID>55</TXID>
       <PATH>/hard-lease-recovery-test</PATH>
-      <NUMBLOCKS>1</NUMBLOCKS>
       <BLOCK>
         <BLOCK_ID>-357061736603024522</BLOCK_ID>
-        <BLOCK_NUM_BYTES>0</BLOCK_NUM_BYTES>
-        <BLOCK_GENERATION_STAMP>1016</BLOCK_GENERATION_STAMP>
+        <NUM_BYTES>0</NUM_BYTES>
+        <GENERATION_STAMP>1016</GENERATION_STAMP>
       </BLOCK>
     </DATA>
-    <CHECKSUM>-1794222801</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>10</OPCODE>
+    <OPCODE>OP_SET_GENSTAMP</OPCODE>
     <DATA>
-      <TRANSACTION_ID>56</TRANSACTION_ID>
-      <GENERATION_STAMP>1017</GENERATION_STAMP>
+      <TXID>56</TXID>
+      <GENSTAMP>1017</GENSTAMP>
     </DATA>
-    <CHECKSUM>-2123999915</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>22</OPCODE>
+    <OPCODE>OP_REASSIGN_LEASE</OPCODE>
     <DATA>
-      <TRANSACTION_ID>57</TRANSACTION_ID>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-2143415023_1</CLIENT_NAME>
+      <TXID>57</TXID>
+      <LEASEHOLDER>DFSClient_NONMAPREDUCE_-2143415023_1</LEASEHOLDER>
       <PATH>/hard-lease-recovery-test</PATH>
-      <CLIENT_NAME>HDFS_NameNode</CLIENT_NAME>
+      <NEWHOLDER>HDFS_NameNode</NEWHOLDER>
     </DATA>
-    <CHECKSUM>-1841690515</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>9</OPCODE>
+    <OPCODE>OP_CLOSE</OPCODE>
     <DATA>
-      <TRANSACTION_ID>58</TRANSACTION_ID>
+      <TXID>58</TXID>
+      <LENGTH>0</LENGTH>
       <PATH>/hard-lease-recovery-test</PATH>
       <REPLICATION>1</REPLICATION>
       <MTIME>1330405688726</MTIME>
       <ATIME>1330405686084</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <NUMBLOCKS>1</NUMBLOCKS>
+      <CLIENT_NAME></CLIENT_NAME>
+      <CLIENT_MACHINE></CLIENT_MACHINE>
       <BLOCK>
         <BLOCK_ID>-357061736603024522</BLOCK_ID>
-        <BLOCK_NUM_BYTES>11</BLOCK_NUM_BYTES>
-        <BLOCK_GENERATION_STAMP>1017</BLOCK_GENERATION_STAMP>
+        <NUM_BYTES>11</NUM_BYTES>
+        <GENERATION_STAMP>1017</GENERATION_STAMP>
       </BLOCK>
       <PERMISSION_STATUS>
         <USERNAME>todd</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
-        <FS_PERMISSIONS>420</FS_PERMISSIONS>
+        <MODE>420</MODE>
       </PERMISSION_STATUS>
     </DATA>
-    <CHECKSUM>-218102037</CHECKSUM>
-  </RECORD>
-  <RECORD>
-    <OPCODE>23</OPCODE>
-    <DATA>
-      <TRANSACTION_ID>59</TRANSACTION_ID>
-    </DATA>
-    <CHECKSUM>-1616653774</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>-1</OPCODE>
+    <OPCODE>OP_END_LOG_SEGMENT</OPCODE>
     <DATA>
+      <TXID>59</TXID>
     </DATA>
   </RECORD>
 </EDITS>

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

@@ -201,6 +201,9 @@ Release 2.0.0 - UNRELEASED
     (Colin Patrick McCabe via eli)
 
     MAPREDUCE-4098. TestMRApps testSetClasspath fails (tucu)
+
+    MAPREDUCE-4097. tools testcases fail because missing mrapp-generated-classpath 
+    file in classpath (rvs via tucu)
  
 Release 0.23.3 - UNRELEASED
 
@@ -254,6 +257,13 @@ Release 0.23.3 - UNRELEASED
     directory (John George via bobby)
 
     MAPREDUCE-3650. testGetTokensForHftpFS() fails (Ravi Prakash via bobby)
+
+    MAPREDUCE-3621. TestDBJob and TestDataDrivenDBInputFormat ant tests fail
+    (Ravi Prakash via tgraves)
+
+    MAPREDUCE-4073. CS assigns multiple off-switch containers when using
+    multi-level-queues (Siddharth Seth via bobby)
+
 Release 0.23.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 0 - 15
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/pom.xml

@@ -92,21 +92,6 @@
           </excludes>
         </configuration>
       </plugin>
-      <plugin>
-        <artifactId>maven-dependency-plugin</artifactId>
-        <executions>
-          <execution>
-            <id>build-classpath</id>
-            <phase>generate-sources</phase>
-            <goals>
-              <goal>build-classpath</goal>
-            </goals>
-            <configuration>
-              <outputFile>target/classes/mrapp-generated-classpath</outputFile>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
       <plugin>
         <artifactId>maven-antrun-plugin</artifactId>
         <executions>

+ 6 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/pom.xml

@@ -88,6 +88,12 @@
       <scope>test</scope>
       <type>test-jar</type>
     </dependency>
+    <dependency>
+      <groupId>org.hsqldb</groupId>
+      <artifactId>hsqldb</artifactId>
+      <version>2.0.0</version>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
   <build>

+ 5 - 5
hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapreduce/lib/db/TestDataDrivenDBInputFormat.java → hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDataDrivenDBInputFormat.java

@@ -25,7 +25,7 @@ import java.io.IOException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.examples.DBCountPageView;
+//import org.apache.hadoop.examples.DBCountPageView;
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.NullWritable;
@@ -36,7 +36,7 @@ import org.apache.hadoop.mapreduce.lib.db.*;
 import org.apache.hadoop.mapreduce.lib.input.*;
 import org.apache.hadoop.mapreduce.lib.output.*;
 import org.apache.hadoop.util.StringUtils;
-import org.hsqldb.Server;
+import org.hsqldb.server.Server;
 
 /**
  * Test aspects of DataDrivenDBInputFormat
@@ -47,9 +47,9 @@ public class TestDataDrivenDBInputFormat extends HadoopTestCase {
       TestDataDrivenDBInputFormat.class);
 
   private static final String DB_NAME = "dddbif";
-  private static final String DB_URL = 
+  private static final String DB_URL =
     "jdbc:hsqldb:hsql://localhost/" + DB_NAME;
-  private static final String DRIVER_CLASS = "org.hsqldb.jdbcDriver";
+  private static final String DRIVER_CLASS = "org.hsqldb.jdbc.JDBCDriver";
 
   private Server server;
   private Connection connection;
@@ -181,7 +181,7 @@ public class TestDataDrivenDBInputFormat extends HadoopTestCase {
     }
 
     // Create the table.
-    s.executeUpdate("CREATE TABLE " + DATE_TABLE + "(" + COL + " TIMESTAMP)");
+    s.executeUpdate("CREATE TABLE " + DATE_TABLE + "(" + COL + " DATE)");
     s.executeUpdate("INSERT INTO " + DATE_TABLE + " VALUES('2010-04-01')");
     s.executeUpdate("INSERT INTO " + DATE_TABLE + " VALUES('2010-04-02')");
     s.executeUpdate("INSERT INTO " + DATE_TABLE + " VALUES('2010-05-01')");

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

@@ -86,10 +86,6 @@
           <groupId>org.mortbay.jetty</groupId>
           <artifactId>jsp-2.1-jetty</artifactId>
         </exclusion>
-        <exclusion>
-          <groupId>hsqldb</groupId>
-          <artifactId>hsqldb</artifactId>
-        </exclusion>
       </exclusions>
     </dependency>
 

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

@@ -92,6 +92,11 @@
        <artifactId>hadoop-mapreduce-client-hs</artifactId>
        <scope>test</scope>
      </dependency>
+     <dependency>
+       <groupId>org.hsqldb</groupId>
+       <artifactId>hsqldb</artifactId>
+       <version>2.0.0</version>
+     </dependency>
   </dependencies>
   
   <build>

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/DBCountPageView.java

@@ -49,7 +49,7 @@ import org.apache.hadoop.mapreduce.lib.reduce.LongSumReducer;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
-import org.hsqldb.Server;
+import org.hsqldb.server.Server;
 
 /**
  * This is a demonstrative program, which uses DBInputFormat for reading
@@ -78,7 +78,7 @@ public class DBCountPageView extends Configured implements Tool {
   
   private static final String DB_URL = 
     "jdbc:hsqldb:hsql://localhost/URLAccess";
-  private static final String DRIVER_CLASS = "org.hsqldb.jdbcDriver";
+  private static final String DRIVER_CLASS = "org.hsqldb.jdbc.JDBCDriver";
   
   private Server server;
   

+ 2 - 2
hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapreduce/lib/db/TestDBJob.java → hadoop-mapreduce-project/hadoop-mapreduce-examples/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDBJob.java

@@ -30,10 +30,10 @@ public class TestDBJob extends HadoopTestCase {
   public TestDBJob() throws IOException {
     super(LOCAL_MR, LOCAL_FS, 3, 1);
   }
-  
+
   public void testRun() throws Exception {
     DBCountPageView testDriver = new DBCountPageView();
     ToolRunner.run(createJobConf(), testDriver, new String[0]);
   }
-  
+
 }

+ 5 - 1
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSAssignment.java

@@ -26,7 +26,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 @Unstable
 public class CSAssignment {
   final private Resource resource;
-  final private NodeType type;
+  private NodeType type;
   
   public CSAssignment(Resource resource, NodeType type) {
     this.resource = resource;
@@ -41,6 +41,10 @@ public class CSAssignment {
     return type;
   }
   
+  public void setType(NodeType type) {
+    this.type = type;
+  }
+  
   @Override
   public String toString() {
     return resource.getMemory() + ":" + type;

+ 5 - 8
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java

@@ -519,7 +519,6 @@ public class ParentQueue implements CSQueue {
       Resource clusterResource, SchedulerNode node) {
     CSAssignment assignment = 
         new CSAssignment(Resources.createResource(0), NodeType.NODE_LOCAL);
-    boolean assignedOffSwitch = false;
     
     while (canAssign(node)) {
       if (LOG.isDebugEnabled()) {
@@ -535,7 +534,7 @@ public class ParentQueue implements CSQueue {
       // Schedule
       CSAssignment assignedToChild = 
           assignContainersToChildQueues(clusterResource, node);
-      assignedOffSwitch = (assignedToChild.getType() == NodeType.OFF_SWITCH);
+      assignment.setType(assignedToChild.getType());
       
       // Done if no child-queue assigned anything
       if (Resources.greaterThan(assignedToChild.getResource(), 
@@ -566,15 +565,13 @@ public class ParentQueue implements CSQueue {
 
       // Do not assign more than one container if this isn't the root queue
       // or if we've already assigned an off-switch container
-      if (rootQueue) {
-        if (assignedOffSwitch) {
-          if (LOG.isDebugEnabled()) {
+      if (!rootQueue || assignment.getType() == NodeType.OFF_SWITCH) {
+        if (LOG.isDebugEnabled()) {
+          if (rootQueue && assignment.getType() == NodeType.OFF_SWITCH) {
             LOG.debug("Not assigning more than one off-switch container," +
-            		" assignments so far: " + assignment);
+                " assignments so far: " + assignment);
           }
-          break;
         }
-      } else {
         break;
       }
     } 

+ 65 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java

@@ -499,6 +499,71 @@ public class TestParentQueue {
 
   }
   
+  @Test
+  public void testOffSwitchSchedulingMultiLevelQueues() throws Exception {
+    // Setup queue configs
+    setupMultiLevelQueues(csConf);
+    //B3
+    Map<String, CSQueue> queues = new HashMap<String, CSQueue>();
+    CSQueue root = 
+        CapacityScheduler.parseQueue(csContext, csConf, null, 
+            CapacitySchedulerConfiguration.ROOT, queues, queues, 
+            CapacityScheduler.queueComparator, 
+            CapacityScheduler.applicationComparator,
+            TestUtils.spyHook);
+
+    // Setup some nodes
+    final int memoryPerNode = 10;
+    final int numNodes = 2;
+    
+    SchedulerNode node_0 = 
+        TestUtils.getMockNode("host_0", DEFAULT_RACK, 0, memoryPerNode*GB);
+    SchedulerNode node_1 = 
+        TestUtils.getMockNode("host_1", DEFAULT_RACK, 0, memoryPerNode*GB);
+    
+    final Resource clusterResource = 
+        Resources.createResource(numNodes * (memoryPerNode*GB));
+    when(csContext.getNumClusterNodes()).thenReturn(numNodes);
+
+    // Start testing
+    LeafQueue b3 = (LeafQueue)queues.get(B3);
+    LeafQueue b2 = (LeafQueue)queues.get(B2);
+    
+    // Simulate B3 returning a container on node_0
+    stubQueueAllocation(b2, clusterResource, node_0, 0*GB, NodeType.OFF_SWITCH);
+    stubQueueAllocation(b3, clusterResource, node_0, 1*GB, NodeType.OFF_SWITCH);
+    root.assignContainers(clusterResource, node_0);
+    verifyQueueMetrics(b2, 0*GB, clusterResource);
+    verifyQueueMetrics(b3, 1*GB, clusterResource);
+    
+    // Now, B2 should get the scheduling opportunity since B2=0G/2G, B3=1G/7G
+    // also, B3 gets a scheduling opportunity since B2 allocates RACK_LOCAL
+    stubQueueAllocation(b2, clusterResource, node_1, 1*GB, NodeType.RACK_LOCAL);
+    stubQueueAllocation(b3, clusterResource, node_1, 1*GB, NodeType.OFF_SWITCH);
+    root.assignContainers(clusterResource, node_1);
+    InOrder allocationOrder = inOrder(b2, b3);
+    allocationOrder.verify(b2).assignContainers(eq(clusterResource), 
+        any(SchedulerNode.class));
+    allocationOrder.verify(b3).assignContainers(eq(clusterResource), 
+        any(SchedulerNode.class));
+    verifyQueueMetrics(b2, 1*GB, clusterResource);
+    verifyQueueMetrics(b3, 2*GB, clusterResource);
+    
+    // Now, B3 should get the scheduling opportunity 
+    // since B2 has 1/2G while B3 has 2/7G, 
+    // However, since B3 returns off-switch, B2 won't get an opportunity
+    stubQueueAllocation(b2, clusterResource, node_0, 1*GB, NodeType.NODE_LOCAL);
+    stubQueueAllocation(b3, clusterResource, node_0, 1*GB, NodeType.OFF_SWITCH);
+    root.assignContainers(clusterResource, node_0);
+    allocationOrder = inOrder(b3, b2);
+    allocationOrder.verify(b3).assignContainers(eq(clusterResource), 
+        any(SchedulerNode.class));
+    allocationOrder.verify(b2).assignContainers(eq(clusterResource), 
+        any(SchedulerNode.class));
+    verifyQueueMetrics(b2, 1*GB, clusterResource);
+    verifyQueueMetrics(b3, 3*GB, clusterResource);
+
+  }
 
   public boolean hasQueueACL(List<QueueUserACLInfo> aclInfos, QueueACL acl, String qName) {
     for (QueueUserACLInfo aclInfo : aclInfos) {

+ 0 - 4
hadoop-mapreduce-project/hadoop-yarn/pom.xml

@@ -55,10 +55,6 @@
           <groupId>org.mortbay.jetty</groupId>
           <artifactId>jsp-2.1-jetty</artifactId>
         </exclusion>
-        <exclusion>
-          <groupId>hsqldb</groupId>
-          <artifactId>hsqldb</artifactId>
-        </exclusion>
       </exclusions>
     </dependency>
 

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

@@ -100,10 +100,6 @@
           <groupId>org.mortbay.jetty</groupId>
           <artifactId>jsp-2.1-jetty</artifactId>
         </exclusion>
-        <exclusion>
-          <groupId>hsqldb</groupId>
-          <artifactId>hsqldb</artifactId>
-        </exclusion>
       </exclusions>
     </dependency>
 

+ 20 - 5
hadoop-project/pom.xml

@@ -507,11 +507,6 @@
         <artifactId>commons-configuration</artifactId>
         <version>1.6</version>
       </dependency>
-      <dependency>
-        <groupId>hsqldb</groupId>
-        <artifactId>hsqldb</artifactId>
-        <version>1.8.0.7</version>
-      </dependency>
       <dependency>
         <groupId>org.slf4j</groupId>
         <artifactId>slf4j-api</artifactId>
@@ -684,6 +679,11 @@
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-jar-plugin</artifactId>
           <version>2.3.1</version>
+          <configuration>
+            <excludes>
+              <exclude>mrapp-generated-classpath</exclude>
+            </excludes>
+          </configuration>
         </plugin>
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
@@ -788,6 +788,21 @@
           </execution>
         </executions>
       </plugin>
+      <plugin>
+        <artifactId>maven-dependency-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>build-classpath</id>
+            <phase>generate-sources</phase>
+            <goals>
+              <goal>build-classpath</goal>
+            </goals>
+            <configuration>
+              <outputFile>target/classes/mrapp-generated-classpath</outputFile>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-surefire-plugin</artifactId>