Browse Source

HDFS-3050. svn merge -c 1309629 from trunk

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1309630 13f79535-47bb-0310-9956-ffa450edef68
Eli Collins 13 years ago
parent
commit
6896eea01b
26 changed files with 1784 additions and 2444 deletions
  1. 1 1
      hadoop-hdfs-project/dev-support/test-patch.properties
  2. 3 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  3. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java
  4. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java
  5. 580 24
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
  6. 17 70
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/BinaryEditsVisitor.java
  7. 0 88
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsElement.java
  8. 0 74
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsLoader.java
  9. 0 608
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsLoaderCurrent.java
  10. 0 175
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsVisitor.java
  11. 0 55
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsVisitorFactory.java
  12. 83 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsBinaryLoader.java
  13. 62 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsLoader.java
  14. 54 50
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsViewer.java
  15. 21 29
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsVisitor.java
  16. 77 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsVisitorFactory.java
  17. 256 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsXmlLoader.java
  18. 26 106
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/StatisticsEditsVisitor.java
  19. 40 19
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TeeOutputStream.java
  20. 0 109
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TextEditsVisitor.java
  21. 0 401
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/Tokenizer.java
  22. 65 113
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/XmlEditsVisitor.java
  23. 0 140
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/XmlTokenizer.java
  24. 171 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/XMLUtils.java
  25. 9 14
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TestOfflineEditsViewer.java
  26. 315 364
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml

+ 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

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

@@ -216,6 +216,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-2477. Optimize computing the diff between a block report and the

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

@@ -35,7 +35,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;
@@ -69,7 +69,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);
+  }
 }

+ 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();
+    }
+  }
+}

+ 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;

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