You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by sz...@apache.org on 2011/01/21 23:08:05 UTC

svn commit: r1062045 [2/2] - in /hadoop/hdfs/trunk: ./ bin/ src/docs/src/documentation/content/xdocs/ src/java/org/apache/hadoop/hdfs/server/namenode/ src/java/org/apache/hadoop/hdfs/server/protocol/ src/java/org/apache/hadoop/hdfs/tools/offlineEditsVi...

Added: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsViewer.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsViewer.java?rev=1062045&view=auto
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsViewer.java (added)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsViewer.java Fri Jan 21 22:08:04 2011
@@ -0,0 +1,207 @@
+/**
+ * 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.BufferedInputStream;
+import java.io.DataInputStream;
+import java.io.EOFException;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.commons.cli.PosixParser;
+
+/**
+ * This class implements an offline edits viewer, tool that
+ * can be used to view edit logs.
+ */
+@InterfaceAudience.Private
+@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.
+   */  
+  private void printHelp() {
+    String summary =
+      "Usage: bin/hdfs oev [OPTIONS] -i INPUT_FILE -o OUTPUT_FILE\n" +
+      "Offline edits viewer\n" +
+      "Parse a Hadoop edits log file INPUT_FILE and save results\n" +
+      "in OUTPUT_FILE.\n" +
+      "Required command line arguments:\n" +
+      "-i,--inputFile <arg>   edits file to process, xml (case\n" +
+      "                       insensitive) extension means XML format,\n" +
+      "                       any other filename means binary format\n" +
+      "-o,--outputFile <arg>  Name of output file. If the specified\n" +
+      "                       file exists, it will be overwritten,\n" +
+      "                       format of the file is determined\n" +
+      "                       by -p option\n" +
+      "\n" + 
+      "Optional command line arguments:\n" +
+      "-p,--processor <arg>   Select which type of processor to apply\n" +
+      "                       against image file, currently supported\n" +
+      "                       processors are: binary (native binary format\n" +
+      "                       that Hadoop uses), xml (default, XML\n" +
+      "                       format), stats (prints statistics about\n" +
+      "                       edits file)\n" +
+      "-h,--help              Display usage information and exit\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" +
+      "                       image files this will dramatically increase\n" +
+      "                       processing time (default is false).\n";
+
+
+    System.out.println(summary);
+    System.out.println();
+    ToolRunner.printGenericCommandUsage(System.out);
+  }
+
+  /**
+   * Build command-line options and descriptions
+   *
+   * @return command line options
+   */
+  public static Options buildOptions() {
+    Options options = new Options();
+
+    // Build in/output file arguments, which are required, but there is no 
+    // addOption method that can specify this
+    OptionBuilder.isRequired();
+    OptionBuilder.hasArgs();
+    OptionBuilder.withLongOpt("outputFilename");
+    options.addOption(OptionBuilder.create("o"));
+    
+    OptionBuilder.isRequired();
+    OptionBuilder.hasArgs();
+    OptionBuilder.withLongOpt("inputFilename");
+    options.addOption(OptionBuilder.create("i"));
+    
+    options.addOption("p", "processor", true, "");
+    options.addOption("v", "verbose", false, "");
+    options.addOption("h", "help", false, "");
+
+    return options;
+  }
+
+  /**
+   * Main entry point for ToolRunner (see ToolRunner docs)
+   *
+   * @param argv The parameters passed to this program.
+   * @return 0 on success, non zero on error.
+   */
+  @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) {
+      System.out.println(
+        "Error parsing command-line options: " + e.getMessage());
+      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 + "]");
+    }
+
+    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;
+  }
+
+  /**
+   * main() runs the offline edits viewer using ToolRunner
+   *
+   * @param argv Command line parameters.
+   */
+  public static void main(String[] argv) throws Exception {
+    int res = ToolRunner.run(new OfflineEditsViewer(), argv);
+    System.exit(res);
+  }
+}

Added: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/StatisticsEditsVisitor.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/StatisticsEditsVisitor.java?rev=1062045&view=auto
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/StatisticsEditsVisitor.java (added)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/StatisticsEditsVisitor.java Fri Jan 21 22:08:04 2011
@@ -0,0 +1,201 @@
+/**
+ * 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 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.FSEditLogOpCodes;
+
+/**
+ * StatisticsEditsVisitor implements text version of EditsVisitor
+ * that aggregates counts of op codes processed
+ *
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class StatisticsEditsVisitor extends EditsVisitor {
+  private boolean printToScreen = false;
+  private boolean okToWrite = false;
+  final private FileWriter fw;
+
+  public 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.
+   *
+   * @param filename Name of file to write output to
+   * @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;
+  }
+
+  /**
+   * Start the visitor (initialization)
+   */
+  @Override
+  void start() throws IOException {
+    // nothing to do
+  }
+  
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.hdfs.tools.offlineEditsViewer.EditsVisitor#finish()
+   */
+  @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());
+      }
+    }
+    return value;
+  }
+
+  /**
+   * 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;
+    }
+  }
+
+  /**
+   * Increment the op code counter
+   *
+   * @param opCode opCode for which to increment count
+   */
+  private void incrementOpCodeCount(FSEditLogOpCodes opCode) {
+    if(!opCodeCount.containsKey(opCode)) {
+      opCodeCount.put(opCode, 0L);
+    }
+    Long newValue = opCodeCount.get(opCode) + 1;
+    opCodeCount.put(opCode, newValue);
+  }
+
+  /**
+   * Get statistics
+   *
+   * @return statistics, map of counts per opCode
+   */
+  public Map<FSEditLogOpCodes, Long> getStatistics() {
+    return opCodeCount;
+  }
+
+  /**
+   * Get the statistics in string format, suitable for printing
+   *
+   * @return statistics in in string format, suitable for printing
+   */
+  public String getStatisticsString() {
+    StringBuffer sb = new StringBuffer();
+    for(FSEditLogOpCodes opCode : FSEditLogOpCodes.values()) {
+      sb.append(String.format(
+        "    %-30.30s (%3d): %d%n",
+        opCode,
+        opCode.getOpCode(),
+        opCodeCount.get(opCode)));
+    }
+    return sb.toString();
+  }
+}

Added: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TextEditsVisitor.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TextEditsVisitor.java?rev=1062045&view=auto
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TextEditsVisitor.java (added)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TextEditsVisitor.java Fri Jan 21 22:08:04 2011
@@ -0,0 +1,109 @@
+/**
+ * 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;
+    }
+  }
+}

Added: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/Tokenizer.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/Tokenizer.java?rev=1062045&view=auto
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/Tokenizer.java (added)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/Tokenizer.java Fri Jan 21 22:08:04 2011
@@ -0,0 +1,397 @@
+/**
+ * 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();
+    }
+
+    @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;
+}

Added: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TokenizerFactory.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TokenizerFactory.java?rev=1062045&view=auto
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TokenizerFactory.java (added)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TokenizerFactory.java Fri Jan 21 22:08:04 2011
@@ -0,0 +1,48 @@
+/**
+ * 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.util.regex.Pattern;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * TokenizerFactory for different implementations of Tokenizer
+ *
+ */
+@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);
+    }
+  }
+
+}

Added: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/XmlEditsVisitor.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/XmlEditsVisitor.java?rev=1062045&view=auto
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/XmlEditsVisitor.java (added)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/XmlEditsVisitor.java Fri Jan 21 22:08:04 2011
@@ -0,0 +1,170 @@
+/**
+ * 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.util.LinkedList;
+
+import org.apache.hadoop.hdfs.tools.offlineImageViewer.DepthCounter;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * An XmlEditsVisitor walks over an EditLog structure and writes out
+ * an equivalent XML document that contains the EditLog's components.
+ */
+@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);
+  }
+
+  /**
+   * 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)
+   */
+  public XmlEditsVisitor(String filename,
+    Tokenizer tokenizer,
+    boolean printToScreen) throws IOException {
+
+    super(filename, tokenizer, printToScreen);
+  }
+
+  /**
+   * 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;
+  }
+
+  /**
+   * 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();
+  }
+
+  /**
+   * Leave enclosing element
+   */
+  @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");
+    }
+  }
+
+  // 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 {
+    try {
+      write(indents[depthCounter.getLevel()]);
+    } catch (IndexOutOfBoundsException e) {
+      // unlikely needed so can be slow
+      for(int i = 0; i < depthCounter.getLevel(); i++)
+        write("  ");
+    }
+   
+  }
+}

Added: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/XmlTokenizer.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/XmlTokenizer.java?rev=1062045&view=auto
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/XmlTokenizer.java (added)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/XmlTokenizer.java Fri Jan 21 22:08:04 2011
@@ -0,0 +1,140 @@
+/**
+ * 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;
+  }
+}

Added: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/DepthCounter.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/DepthCounter.java?rev=1062045&view=auto
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/DepthCounter.java (added)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/DepthCounter.java Fri Jan 21 22:08:04 2011
@@ -0,0 +1,36 @@
+/**
+ * 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.offlineImageViewer;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Utility class for tracking descent into the structure of the
+ * Visitor class (ImageVisitor, EditsVisitor etc.)
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class DepthCounter {
+  private int depth = 0;
+
+  public void incLevel() { depth++; }
+  public void decLevel() { if(depth >= 1) depth--; }
+  public int  getLevel() { return depth; }
+}
+

Modified: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageVisitor.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageVisitor.java?rev=1062045&r1=1062044&r2=1062045&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageVisitor.java (original)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageVisitor.java Fri Jan 21 22:08:04 2011
@@ -75,17 +75,6 @@ abstract class ImageVisitor {
   }
   
   /**
-   * Utility class for tracking descent into the structure of the fsimage.
-   */
-  static protected class DepthCounter {
-    private int depth = 0;
-
-    public void incLevel() { depth++; }
-    public void decLevel() { if(depth >= 1) depth--; }
-    public int  getLevel() { return depth; }
-  }
-
-  /**
    * Begin visiting the fsimage structure.  Opportunity to perform
    * any initialization necessary for the implementing visitor.
    */

Modified: hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/DFSTestUtil.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/DFSTestUtil.java?rev=1062045&r1=1062044&r2=1062045&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/DFSTestUtil.java (original)
+++ hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/DFSTestUtil.java Fri Jan 21 22:08:04 2011
@@ -23,6 +23,8 @@ import java.io.ByteArrayOutputStream;
 import java.io.File;
 import java.io.FileReader;
 import java.io.IOException;
+import java.io.FileInputStream;
+import java.io.DataInputStream;
 import java.net.URL;
 import java.net.URLConnection;
 import java.security.PrivilegedExceptionAction;
@@ -376,4 +378,15 @@ public class DFSTestUtil {
   public static Statistics getStatistics(FileSystem fs) {
     return FileSystem.getStatistics(fs.getUri().getScheme(), fs.getClass());
   }
+
+  /**
+   * Load file into byte[]
+   */
+  public static byte[] loadFile(String filename) throws IOException {
+    File file = new File(filename);
+    DataInputStream in = new DataInputStream(new FileInputStream(file));
+    byte[] content = new byte[(int)file.length()];
+    in.readFully(content);
+    return content;
+  }
 }

Added: hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/OfflineEditsViewerHelper.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/OfflineEditsViewerHelper.java?rev=1062045&view=auto
==============================================================================
--- hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/OfflineEditsViewerHelper.java (added)
+++ hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/OfflineEditsViewerHelper.java Fri Jan 21 22:08:04 2011
@@ -0,0 +1,224 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.server.namenode;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.io.File;
+import java.security.PrivilegedExceptionAction;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.common.Util;
+import org.apache.hadoop.hdfs.server.namenode.FSImage;
+import org.apache.hadoop.hdfs.server.namenode.FSImage.NameNodeDirType;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
+import org.apache.hadoop.hdfs.protocol.FSConstants;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.Options.Rename;
+
+/**
+ * OfflineEditsViewerHelper is a helper class for TestOfflineEditsViewer,
+ * it performs NN operations that generate all op codes
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class OfflineEditsViewerHelper {
+
+  private static final Log LOG = 
+    LogFactory.getLog(OfflineEditsViewerHelper.class);
+
+    long           blockSize = 512;
+    MiniDFSCluster cluster   = null;
+    Configuration  config    = new Configuration();
+
+  /**
+   * Generates edits with all op codes and returns the edits filename
+   *
+   * @param dfsDir DFS directory (where to setup MiniDFS cluster)
+   * @param editsFilename where to copy the edits
+   */
+  public String generateEdits() throws IOException {
+    runOperations();
+    return getEditsFilename();
+  }
+
+  /**
+   * Get edits filename
+   *
+   * @return edits file name for cluster
+   */
+  private String getEditsFilename() throws IOException {
+    FSImage image = cluster.getNameNode().getFSImage();
+    // it was set up to only have ONE StorageDirectory
+    Iterator<StorageDirectory> it = image.dirIterator(NameNodeDirType.EDITS);
+    StorageDirectory sd = it.next();
+    return image.getEditFile(sd).getAbsolutePath();
+  }
+
+  /**
+   * Sets up a MiniDFSCluster, configures it to create one edits file,
+   * starts DelegationTokenSecretManager (to get security op codes)
+   *
+   * @param dfsDir DFS directory (where to setup MiniDFS cluster)
+   */
+  public void startCluster(String dfsDir) throws IOException {
+
+    // same as manageDfsDirs but only one edits file instead of two
+    config.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
+      Util.fileAsURI(new File(dfsDir, "name")).toString());
+    config.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_DIR_KEY,
+      Util.fileAsURI(new File(dfsDir, "namesecondary1")).toString());
+    // blocksize for concat (file size must be multiple of blocksize)
+    config.setLong("dfs.blocksize", blockSize);
+    // for security to work (fake JobTracker user)
+    config.set("hadoop.security.auth_to_local",
+      "RULE:[2:$1@$0](JobTracker@.*FOO.COM)s/@.*//" + "DEFAULT");
+    cluster =
+      new MiniDFSCluster.Builder(config).manageNameDfsDirs(false).build();
+    cluster.waitClusterUp();
+    cluster.getNamesystem().getDelegationTokenSecretManager().startThreads();
+  }
+
+  /**
+   * Shutdown the cluster
+   */
+  public void shutdownCluster() throws IOException {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * Run file operations to create edits for all op codes
+   * to be tested.
+   *
+   * the following op codes are deprecated and therefore not tested:
+   *
+   * OP_DATANODE_ADD    ( 5)
+   * OP_DATANODE_REMOVE ( 6)
+   * OP_SET_NS_QUOTA    (11)
+   * OP_CLEAR_NS_QUOTA  (12)
+   */
+  private void runOperations() throws IOException {
+
+    LOG.info("Creating edits by performing fs operations");
+    // no check, if it's not it throws an exception which is what we want
+    DistributedFileSystem dfs =
+      (DistributedFileSystem)cluster.getFileSystem();
+    FileContext fc = FileContext.getFileContext(cluster.getURI(), config);
+    // OP_ADD 0, OP_SET_GENSTAMP 10
+    Path pathFileCreate = new Path("/file_create");
+    FSDataOutputStream s = dfs.create(pathFileCreate);
+    // OP_CLOSE 9
+    s.close();
+    // OP_RENAME_OLD 1
+    Path pathFileMoved = new Path("/file_moved");
+    dfs.rename(pathFileCreate, pathFileMoved);
+    // OP_DELETE 2
+    dfs.delete(pathFileMoved, false);
+    // OP_MKDIR 3
+    Path pathDirectoryMkdir = new Path("/directory_mkdir");
+    dfs.mkdirs(pathDirectoryMkdir);
+    // OP_SET_REPLICATION 4
+    s = dfs.create(pathFileCreate);
+    s.close();
+    dfs.setReplication(pathFileCreate, (short)1);
+    // OP_SET_PERMISSIONS 7
+    Short permission = 0777;
+    dfs.setPermission(pathFileCreate, new FsPermission(permission));
+    // OP_SET_OWNER 8
+    dfs.setOwner(pathFileCreate, new String("newOwner"), null);
+    // OP_CLOSE 9 see above
+    // OP_SET_GENSTAMP 10 see above
+    // OP_SET_NS_QUOTA 11 obsolete
+    // OP_CLEAR_NS_QUOTA 12 obsolete
+    // OP_TIMES 13
+    long mtime = 1285195527000L; // Wed, 22 Sep 2010 22:45:27 GMT
+    long atime = mtime;
+    dfs.setTimes(pathFileCreate, mtime, atime);
+    // OP_SET_QUOTA 14
+    dfs.setQuota(pathDirectoryMkdir, 1000L, FSConstants.QUOTA_DONT_SET);
+    // OP_RENAME 15
+    fc.rename(pathFileCreate, pathFileMoved, Rename.NONE);
+    // OP_CONCAT_DELETE 16
+    Path   pathConcatTarget = new Path("/file_concat_target");
+    Path[] pathConcatFiles  = new Path[2];
+    pathConcatFiles[0]      = new Path("/file_concat_0");
+    pathConcatFiles[1]      = new Path("/file_concat_1");
+
+    long  length      = blockSize * 3; // multiple of blocksize for concat
+    short replication = 1;
+    long  seed        = 1;
+
+    DFSTestUtil.createFile(dfs, pathConcatTarget, length, replication, seed);
+    DFSTestUtil.createFile(dfs, pathConcatFiles[0], length, replication, seed);
+    DFSTestUtil.createFile(dfs, pathConcatFiles[1], length, replication, seed);
+    dfs.concat(pathConcatTarget, pathConcatFiles);
+    // OP_SYMLINK 17
+    Path pathSymlink = new Path("/file_symlink");
+    fc.createSymlink(pathConcatTarget, pathSymlink, false);
+    // OP_GET_DELEGATION_TOKEN 18
+    final Token<DelegationTokenIdentifier> token =
+      dfs.getDelegationToken("JobTracker");
+    // OP_RENEW_DELEGATION_TOKEN 19
+    // OP_CANCEL_DELEGATION_TOKEN 20
+    // see TestDelegationToken.java
+    // fake the user to renew token for
+    UserGroupInformation longUgi = UserGroupInformation.createRemoteUser(
+      "JobTracker/foo.com@FOO.COM");
+    UserGroupInformation shortUgi = UserGroupInformation.createRemoteUser(
+      "JobTracker");
+    try {
+      longUgi.doAs(new PrivilegedExceptionAction<Object>() {
+        public Object run() throws IOException {
+          final DistributedFileSystem dfs =
+            (DistributedFileSystem) cluster.getFileSystem();
+          dfs.renewDelegationToken(token);
+          dfs.cancelDelegationToken(token);
+          return null;
+        }
+      });
+    } catch(InterruptedException e) {
+      throw new IOException(
+        "renewDelegationToken threw InterruptedException", e);
+    }
+    // OP_UPDATE_MASTER_KEY 21
+    //   done by getDelegationTokenSecretManager().startThreads();
+
+    // sync to disk, otherwise we parse partial edits
+    cluster.getNameNode().getFSImage().getEditLog().logSync();
+  }
+}

Added: hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TestOfflineEditsViewer.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TestOfflineEditsViewer.java?rev=1062045&view=auto
==============================================================================
--- hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TestOfflineEditsViewer.java (added)
+++ hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TestOfflineEditsViewer.java Fri Jan 21 22:08:04 2011
@@ -0,0 +1,274 @@
+/**
+ * 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.File;
+import java.io.FileInputStream;
+import java.io.DataInputStream;
+import java.nio.ByteBuffer;
+import java.util.Map;
+import java.util.HashMap;
+
+import org.junit.Test;
+import org.junit.Before;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+
+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.tools.offlineEditsViewer.EditsVisitor;
+import org.apache.hadoop.hdfs.tools.offlineEditsViewer.XmlEditsVisitor;
+import org.apache.hadoop.hdfs.tools.offlineEditsViewer.BinaryEditsVisitor;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+
+import org.apache.hadoop.hdfs.server.namenode.OfflineEditsViewerHelper;
+
+public class TestOfflineEditsViewer {
+
+  private static final Log LOG = LogFactory.getLog(TestOfflineEditsViewer.class);
+
+  private static final Map<FSEditLogOpCodes, Boolean> obsoleteOpCodes =
+    new HashMap<FSEditLogOpCodes, Boolean>();
+
+  static { initializeObsoleteOpCodes(); }
+
+  private static String buildDir =
+    System.getProperty("test.build.data", "build/test/data");
+
+  private static String cacheDir =
+    System.getProperty("test.cache.data", "build/test/data/cache");
+
+  // to create edits and get edits filename
+  private static final OfflineEditsViewerHelper nnHelper 
+    = new OfflineEditsViewerHelper();
+
+  /**
+   * Initialize obsoleteOpCodes
+   *
+   * Reason for suppressing "deprecation" warnings:
+   *
+   * These are the opcodes that are not used anymore, some
+   * are marked deprecated, we need to include them here to make
+   * sure we exclude them when checking for completness of testing,
+   * that's why the "deprecation" warnings are suppressed.
+   */
+  @SuppressWarnings("deprecation")
+  private static void initializeObsoleteOpCodes() {
+    // these are obsolete
+    obsoleteOpCodes.put(FSEditLogOpCodes.OP_DATANODE_ADD, true);
+    obsoleteOpCodes.put(FSEditLogOpCodes.OP_DATANODE_REMOVE, true);
+    obsoleteOpCodes.put(FSEditLogOpCodes.OP_SET_NS_QUOTA, true);
+    obsoleteOpCodes.put(FSEditLogOpCodes.OP_CLEAR_NS_QUOTA, true);
+    // these are not written to files
+    obsoleteOpCodes.put(FSEditLogOpCodes.OP_JSPOOL_START, true);
+    obsoleteOpCodes.put(FSEditLogOpCodes.OP_CHECKPOINT_TIME, true);
+  }
+
+  /**
+   * Test the OfflineEditsViewer
+   */
+  @Test
+  public void testGenerated() throws IOException {
+
+    LOG.info("START - testing with generated edits");
+
+    nnHelper.startCluster(buildDir + "/dfs/");
+
+    // edits generated by nnHelper (MiniDFSCluster), should have all op codes
+    // binary, XML, reparsed binary
+    String edits          = nnHelper.generateEdits();
+    String editsParsedXml = cacheDir + "/editsParsed.xml";
+    String editsReparsed  = cacheDir + "/editsReparsed";
+
+    // parse to XML then back to binary
+    runOev(edits,          editsParsedXml, "xml");
+    runOev(editsParsedXml, editsReparsed,  "binary");
+
+    // judgment time
+    assertTrue(
+      "Edits " + edits + " should have all op codes",
+      hasAllOpCodes(edits));
+    assertTrue(
+      "Generated edits and reparsed (bin to XML to bin) should be same",
+      filesEqualIgnoreTrailingZeros(edits, editsReparsed));
+
+    // removes edits so do this at the end
+    nnHelper.shutdownCluster();
+
+    LOG.info("END");
+  }
+
+  @Test
+  public void testStored() throws IOException {
+
+    LOG.info("START - testing with stored reference edits");
+
+    // reference edits stored with source code (see build.xml)
+    // binary, XML, reparsed binary
+    String editsStored             = cacheDir + "/editsStored";
+    String editsStoredParsedXml    = cacheDir + "/editsStoredParsed.xml";
+    String editsStoredReparsed     = cacheDir + "/editsStoredReparsed";
+    // reference XML version of editsStored (see build.xml)
+    String editsStoredXml          = cacheDir + "/editsStored.xml";
+      
+    // parse to XML then back to binary
+    runOev(editsStored,             editsStoredParsedXml, "xml");
+    runOev(editsStoredParsedXml,    editsStoredReparsed,  "binary");
+
+    // judgement time
+    assertTrue(
+      "Edits " + editsStored + " should have all op codes",
+      hasAllOpCodes(editsStored));
+    assertTrue(
+      "Reference XML edits and parsed to XML should be same",
+      filesEqual(editsStoredXml, editsStoredParsedXml));
+    assertTrue(
+      "Reference edits and reparsed (bin to XML to bin) should be same",
+      filesEqualIgnoreTrailingZeros(editsStored, editsStoredReparsed));
+
+    LOG.info("END");
+  }
+
+  /**
+   * Run OfflineEditsViewer
+   *
+   * @param inFilename input edits filename
+   * @param outFilename oputput edits filename
+   */
+  private void runOev(String inFilename, String outFilename, String processor)
+    throws IOException {
+
+    LOG.info("Running oev [" + inFilename + "] [" + outFilename + "]");
+
+    OfflineEditsViewer oev = new OfflineEditsViewer();
+    oev.go( EditsVisitorFactory.getEditsVisitor(
+      outFilename,
+      processor,
+      TokenizerFactory.getTokenizer(inFilename),
+      false));
+  }
+
+  /**
+   * Checks that the edits file has all opCodes
+   *
+   * @param filename edits file
+   * @return true is edits (filename) has all opCodes
+   */
+  private boolean hasAllOpCodes(String inFilename) throws IOException {
+    String outFilename = inFilename + ".stats";
+    StatisticsEditsVisitor visitor =
+      (StatisticsEditsVisitor)EditsVisitorFactory.getEditsVisitor(
+        outFilename,
+        "stats",
+        TokenizerFactory.getTokenizer(inFilename),
+        false);
+    OfflineEditsViewer oev = new OfflineEditsViewer();
+    oev.go(visitor);
+    LOG.info("Statistics for " + inFilename + "\n" +
+      visitor.getStatisticsString());
+    
+    boolean hasAllOpCodes = true;
+    for(FSEditLogOpCodes opCode : FSEditLogOpCodes.values()) {
+      // don't need to test obsolete opCodes
+      if(obsoleteOpCodes.containsKey(opCode)) {
+        continue;
+      }
+      Long count = visitor.getStatistics().get(opCode);
+      if((count == null) || (count == 0)) {
+        hasAllOpCodes = false;
+        LOG.info("Opcode " + opCode + " not tested in " + inFilename);
+      }
+    }
+    return hasAllOpCodes;
+  }
+
+  /**
+   * Compare two files, ignore trailing zeros at the end,
+   * for edits log the trailing zeros do not make any difference,
+   * throw exception is the files are not same
+   *
+   * @param filenameSmall first file to compare (doesn't have to be smaller)
+   * @param filenameLarge second file to compare (doesn't have to be larger)
+   */
+  private boolean filesEqualIgnoreTrailingZeros(String filenameSmall,
+    String filenameLarge) throws IOException {
+
+    ByteBuffer small = ByteBuffer.wrap(DFSTestUtil.loadFile(filenameSmall));
+    ByteBuffer large = ByteBuffer.wrap(DFSTestUtil.loadFile(filenameLarge));
+
+    // now correct if it's otherwise
+    if(small.capacity() > large.capacity()) {
+      ByteBuffer tmpByteBuffer = small;
+      small = large;
+      large = tmpByteBuffer;
+      String tmpFilename = filenameSmall;
+      filenameSmall = filenameLarge;
+      filenameLarge = tmpFilename;
+    }
+
+    // compare from 0 to capacity of small
+    // the rest of the large should be all zeros
+    small.position(0);
+    small.limit(small.capacity());
+    large.position(0);
+    large.limit(small.capacity());
+
+    // compares position to limit
+    if(!small.equals(large)) { return false; }
+
+    // everything after limit should be zero 
+    int i = large.limit();
+    large.clear();
+    for(; i < large.capacity(); i++) {
+      if(large.get(i) != 0) {
+        return false;
+      }
+    }
+
+    return true;
+  }
+
+  /**
+   * Compare two files, throw exception is they are not same
+   *
+   * @param filename1 first file to compare
+   * @param filename2 second file to compare
+   */
+  private boolean filesEqual(String filename1,
+    String filename2) throws IOException {
+
+    // make file 1 the small one
+    ByteBuffer bb1 = ByteBuffer.wrap(DFSTestUtil.loadFile(filename1));
+    ByteBuffer bb2 = ByteBuffer.wrap(DFSTestUtil.loadFile(filename2));
+
+    // compare from 0 to capacity
+    bb1.position(0);
+    bb1.limit(bb1.capacity());
+    bb2.position(0);
+    bb2.limit(bb2.capacity());
+
+    return bb1.equals(bb2);
+  }
+}

Added: hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/tools/offlineEditsViewer/editsStored.xml
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/tools/offlineEditsViewer/editsStored.xml?rev=1062045&view=auto
==============================================================================
--- hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/tools/offlineEditsViewer/editsStored.xml (added)
+++ hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/tools/offlineEditsViewer/editsStored.xml Fri Jan 21 22:08:04 2011
@@ -0,0 +1,428 @@
+<?xml version="1.0"?>
+<EDITS>
+  <EDITS_VERSION>-24</EDITS_VERSION>
+  <RECORD>
+    <OPCODE>21</OPCODE>
+    <DATA>
+      <KEY_ID>1</KEY_ID>
+      <KEY_EXPIRY_DATE>1287183164658</KEY_EXPIRY_DATE>
+      <KEY_LENGTH>3</KEY_LENGTH>
+      <KEY_BLOB>drEs</KEY_BLOB>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>21</OPCODE>
+    <DATA>
+      <KEY_ID>2</KEY_ID>
+      <KEY_EXPIRY_DATE>1287183164703</KEY_EXPIRY_DATE>
+      <KEY_LENGTH>3</KEY_LENGTH>
+      <KEY_BLOB>1cGc</KEY_BLOB>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>10</OPCODE>
+    <DATA>
+      <GENERATION_STAMP>1001</GENERATION_STAMP>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>0</OPCODE>
+    <DATA>
+      <LENGTH>5</LENGTH>
+      <PATH>/file_create</PATH>
+      <REPLICATION>1</REPLICATION>
+      <MTIME>1286491964741</MTIME>
+      <ATIME>1286491964741</ATIME>
+      <BLOCKSIZE>512</BLOCKSIZE>
+      <NUMBLOCKS>0</NUMBLOCKS>
+      <PERMISSION_STATUS>
+        <USERNAME>steffl</USERNAME>
+        <GROUPNAME>supergroup</GROUPNAME>
+        <FS_PERMISSIONS>420</FS_PERMISSIONS>
+      </PERMISSION_STATUS>
+      <CLIENT_NAME>DFSClient_471171074</CLIENT_NAME>
+      <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>9</OPCODE>
+    <DATA>
+      <LENGTH>5</LENGTH>
+      <PATH>/file_create</PATH>
+      <REPLICATION>1</REPLICATION>
+      <MTIME>1286491964758</MTIME>
+      <ATIME>1286491964741</ATIME>
+      <BLOCKSIZE>512</BLOCKSIZE>
+      <NUMBLOCKS>0</NUMBLOCKS>
+      <PERMISSION_STATUS>
+        <USERNAME>steffl</USERNAME>
+        <GROUPNAME>supergroup</GROUPNAME>
+        <FS_PERMISSIONS>420</FS_PERMISSIONS>
+      </PERMISSION_STATUS>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>1</OPCODE>
+    <DATA>
+      <LENGTH>3</LENGTH>
+      <SOURCE>/file_create</SOURCE>
+      <DESTINATION>/file_moved</DESTINATION>
+      <TIMESTAMP>1286491964766</TIMESTAMP>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>2</OPCODE>
+    <DATA>
+      <LENGTH>2</LENGTH>
+      <PATH>/file_moved</PATH>
+      <TIMESTAMP>1286491964775</TIMESTAMP>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>3</OPCODE>
+    <DATA>
+      <LENGTH>3</LENGTH>
+      <PATH>/directory_mkdir</PATH>
+      <TIMESTAMP>1286491964783</TIMESTAMP>
+      <ATIME>0</ATIME>
+      <PERMISSION_STATUS>
+        <USERNAME>steffl</USERNAME>
+        <GROUPNAME>supergroup</GROUPNAME>
+        <FS_PERMISSIONS>493</FS_PERMISSIONS>
+      </PERMISSION_STATUS>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>10</OPCODE>
+    <DATA>
+      <GENERATION_STAMP>1002</GENERATION_STAMP>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>0</OPCODE>
+    <DATA>
+      <LENGTH>5</LENGTH>
+      <PATH>/file_create</PATH>
+      <REPLICATION>1</REPLICATION>
+      <MTIME>1286491964796</MTIME>
+      <ATIME>1286491964796</ATIME>
+      <BLOCKSIZE>512</BLOCKSIZE>
+      <NUMBLOCKS>0</NUMBLOCKS>
+      <PERMISSION_STATUS>
+        <USERNAME>steffl</USERNAME>
+        <GROUPNAME>supergroup</GROUPNAME>
+        <FS_PERMISSIONS>420</FS_PERMISSIONS>
+      </PERMISSION_STATUS>
+      <CLIENT_NAME>DFSClient_471171074</CLIENT_NAME>
+      <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>9</OPCODE>
+    <DATA>
+      <LENGTH>5</LENGTH>
+      <PATH>/file_create</PATH>
+      <REPLICATION>1</REPLICATION>
+      <MTIME>1286491964814</MTIME>
+      <ATIME>1286491964796</ATIME>
+      <BLOCKSIZE>512</BLOCKSIZE>
+      <NUMBLOCKS>0</NUMBLOCKS>
+      <PERMISSION_STATUS>
+        <USERNAME>steffl</USERNAME>
+        <GROUPNAME>supergroup</GROUPNAME>
+        <FS_PERMISSIONS>420</FS_PERMISSIONS>
+      </PERMISSION_STATUS>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>4</OPCODE>
+    <DATA>
+      <PATH>/file_create</PATH>
+      <REPLICATION>1</REPLICATION>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>7</OPCODE>
+    <DATA>
+      <PATH>/file_create</PATH>
+      <FS_PERMISSIONS>511</FS_PERMISSIONS>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>8</OPCODE>
+    <DATA>
+      <PATH>/file_create</PATH>
+      <USERNAME>newOwner</USERNAME>
+      <GROUPNAME/>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>13</OPCODE>
+    <DATA>
+      <LENGTH>3</LENGTH>
+      <PATH>/file_create</PATH>
+      <MTIME>1285195527000</MTIME>
+      <ATIME>1285195527000</ATIME>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>14</OPCODE>
+    <DATA>
+      <PATH>/directory_mkdir</PATH>
+      <NS_QUOTA>1000</NS_QUOTA>
+      <DS_QUOTA>-1</DS_QUOTA>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>15</OPCODE>
+    <DATA>
+      <LENGTH>3</LENGTH>
+      <SOURCE>/file_create</SOURCE>
+      <DESTINATION>/file_moved</DESTINATION>
+      <TIMESTAMP>1286491964858</TIMESTAMP>
+      <RENAME_OPTIONS>AA</RENAME_OPTIONS>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>10</OPCODE>
+    <DATA>
+      <GENERATION_STAMP>1003</GENERATION_STAMP>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>0</OPCODE>
+    <DATA>
+      <LENGTH>5</LENGTH>
+      <PATH>/file_concat_target</PATH>
+      <REPLICATION>1</REPLICATION>
+      <MTIME>1286491964873</MTIME>
+      <ATIME>1286491964873</ATIME>
+      <BLOCKSIZE>512</BLOCKSIZE>
+      <NUMBLOCKS>0</NUMBLOCKS>
+      <PERMISSION_STATUS>
+        <USERNAME>steffl</USERNAME>
+        <GROUPNAME>supergroup</GROUPNAME>
+        <FS_PERMISSIONS>420</FS_PERMISSIONS>
+      </PERMISSION_STATUS>
+      <CLIENT_NAME>DFSClient_471171074</CLIENT_NAME>
+      <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>9</OPCODE>
+    <DATA>
+      <LENGTH>5</LENGTH>
+      <PATH>/file_concat_target</PATH>
+      <REPLICATION>1</REPLICATION>
+      <MTIME>1286491965024</MTIME>
+      <ATIME>1286491964873</ATIME>
+      <BLOCKSIZE>512</BLOCKSIZE>
+      <NUMBLOCKS>3</NUMBLOCKS>
+      <BLOCK>
+        <BLOCK_ID>1096087107607101866</BLOCK_ID>
+        <BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
+        <BLOCK_GENERATION_STAMP>1003</BLOCK_GENERATION_STAMP>
+      </BLOCK>
+      <BLOCK>
+        <BLOCK_ID>8798023959648425597</BLOCK_ID>
+        <BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
+        <BLOCK_GENERATION_STAMP>1003</BLOCK_GENERATION_STAMP>
+      </BLOCK>
+      <BLOCK>
+        <BLOCK_ID>4060815343079109399</BLOCK_ID>
+        <BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
+        <BLOCK_GENERATION_STAMP>1003</BLOCK_GENERATION_STAMP>
+      </BLOCK>
+      <PERMISSION_STATUS>
+        <USERNAME>steffl</USERNAME>
+        <GROUPNAME>supergroup</GROUPNAME>
+        <FS_PERMISSIONS>420</FS_PERMISSIONS>
+      </PERMISSION_STATUS>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>10</OPCODE>
+    <DATA>
+      <GENERATION_STAMP>1004</GENERATION_STAMP>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>0</OPCODE>
+    <DATA>
+      <LENGTH>5</LENGTH>
+      <PATH>/file_concat_0</PATH>
+      <REPLICATION>1</REPLICATION>
+      <MTIME>1286491965035</MTIME>
+      <ATIME>1286491965035</ATIME>
+      <BLOCKSIZE>512</BLOCKSIZE>
+      <NUMBLOCKS>0</NUMBLOCKS>
+      <PERMISSION_STATUS>
+        <USERNAME>steffl</USERNAME>
+        <GROUPNAME>supergroup</GROUPNAME>
+        <FS_PERMISSIONS>420</FS_PERMISSIONS>
+      </PERMISSION_STATUS>
+      <CLIENT_NAME>DFSClient_471171074</CLIENT_NAME>
+      <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>9</OPCODE>
+    <DATA>
+      <LENGTH>5</LENGTH>
+      <PATH>/file_concat_0</PATH>
+      <REPLICATION>1</REPLICATION>
+      <MTIME>1286491965093</MTIME>
+      <ATIME>1286491965035</ATIME>
+      <BLOCKSIZE>512</BLOCKSIZE>
+      <NUMBLOCKS>3</NUMBLOCKS>
+      <BLOCK>
+        <BLOCK_ID>85340326229460895</BLOCK_ID>
+        <BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
+        <BLOCK_GENERATION_STAMP>1004</BLOCK_GENERATION_STAMP>
+      </BLOCK>
+      <BLOCK>
+        <BLOCK_ID>4456960998526419279</BLOCK_ID>
+        <BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
+        <BLOCK_GENERATION_STAMP>1004</BLOCK_GENERATION_STAMP>
+      </BLOCK>
+      <BLOCK>
+        <BLOCK_ID>-6161739531018161735</BLOCK_ID>
+        <BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
+        <BLOCK_GENERATION_STAMP>1004</BLOCK_GENERATION_STAMP>
+      </BLOCK>
+      <PERMISSION_STATUS>
+        <USERNAME>steffl</USERNAME>
+        <GROUPNAME>supergroup</GROUPNAME>
+        <FS_PERMISSIONS>420</FS_PERMISSIONS>
+      </PERMISSION_STATUS>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>10</OPCODE>
+    <DATA>
+      <GENERATION_STAMP>1005</GENERATION_STAMP>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>0</OPCODE>
+    <DATA>
+      <LENGTH>5</LENGTH>
+      <PATH>/file_concat_1</PATH>
+      <REPLICATION>1</REPLICATION>
+      <MTIME>1286491965105</MTIME>
+      <ATIME>1286491965105</ATIME>
+      <BLOCKSIZE>512</BLOCKSIZE>
+      <NUMBLOCKS>0</NUMBLOCKS>
+      <PERMISSION_STATUS>
+        <USERNAME>steffl</USERNAME>
+        <GROUPNAME>supergroup</GROUPNAME>
+        <FS_PERMISSIONS>420</FS_PERMISSIONS>
+      </PERMISSION_STATUS>
+      <CLIENT_NAME>DFSClient_471171074</CLIENT_NAME>
+      <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>9</OPCODE>
+    <DATA>
+      <LENGTH>5</LENGTH>
+      <PATH>/file_concat_1</PATH>
+      <REPLICATION>1</REPLICATION>
+      <MTIME>1286491965148</MTIME>
+      <ATIME>1286491965105</ATIME>
+      <BLOCKSIZE>512</BLOCKSIZE>
+      <NUMBLOCKS>3</NUMBLOCKS>
+      <BLOCK>
+        <BLOCK_ID>-3894328423940677915</BLOCK_ID>
+        <BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
+        <BLOCK_GENERATION_STAMP>1005</BLOCK_GENERATION_STAMP>
+      </BLOCK>
+      <BLOCK>
+        <BLOCK_ID>-2833847567910728858</BLOCK_ID>
+        <BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
+        <BLOCK_GENERATION_STAMP>1005</BLOCK_GENERATION_STAMP>
+      </BLOCK>
+      <BLOCK>
+        <BLOCK_ID>-3654781106237722465</BLOCK_ID>
+        <BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
+        <BLOCK_GENERATION_STAMP>1005</BLOCK_GENERATION_STAMP>
+      </BLOCK>
+      <PERMISSION_STATUS>
+        <USERNAME>steffl</USERNAME>
+        <GROUPNAME>supergroup</GROUPNAME>
+        <FS_PERMISSIONS>420</FS_PERMISSIONS>
+      </PERMISSION_STATUS>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>16</OPCODE>
+    <DATA>
+      <LENGTH>4</LENGTH>
+      <CONCAT_TARGET>/file_concat_target</CONCAT_TARGET>
+      <CONCAT_SOURCE>/file_concat_0</CONCAT_SOURCE>
+      <CONCAT_SOURCE>/file_concat_1</CONCAT_SOURCE>
+      <TIMESTAMP>1286491965157</TIMESTAMP>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>17</OPCODE>
+    <DATA>
+      <LENGTH>4</LENGTH>
+      <SOURCE>/file_symlink</SOURCE>
+      <DESTINATION>/file_concat_target</DESTINATION>
+      <MTIME>1286491965168</MTIME>
+      <ATIME>1286491965168</ATIME>
+      <PERMISSION_STATUS>
+        <USERNAME>steffl</USERNAME>
+        <GROUPNAME>supergroup</GROUPNAME>
+        <FS_PERMISSIONS>511</FS_PERMISSIONS>
+      </PERMISSION_STATUS>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>18</OPCODE>
+    <DATA>
+      <T_VERSION>0</T_VERSION>
+      <T_OWNER>steffl</T_OWNER>
+      <T_RENEWER>JobTracker</T_RENEWER>
+      <T_REAL_USER/>
+      <T_ISSUE_DATE>1286491965176</T_ISSUE_DATE>
+      <T_MAX_DATE>1287096765176</T_MAX_DATE>
+      <T_SEQUENCE_NUMBER>1</T_SEQUENCE_NUMBER>
+      <T_MASTER_KEY_ID>2</T_MASTER_KEY_ID>
+      <T_EXPIRY_TIME>1286578365176</T_EXPIRY_TIME>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>19</OPCODE>
+    <DATA>
+      <T_VERSION>0</T_VERSION>
+      <T_OWNER>steffl</T_OWNER>
+      <T_RENEWER>JobTracker</T_RENEWER>
+      <T_REAL_USER/>
+      <T_ISSUE_DATE>1286491965176</T_ISSUE_DATE>
+      <T_MAX_DATE>1287096765176</T_MAX_DATE>
+      <T_SEQUENCE_NUMBER>1</T_SEQUENCE_NUMBER>
+      <T_MASTER_KEY_ID>2</T_MASTER_KEY_ID>
+      <T_EXPIRY_TIME>1286578365198</T_EXPIRY_TIME>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>20</OPCODE>
+    <DATA>
+      <T_VERSION>0</T_VERSION>
+      <T_OWNER>steffl</T_OWNER>
+      <T_RENEWER>JobTracker</T_RENEWER>
+      <T_REAL_USER/>
+      <T_ISSUE_DATE>1286491965176</T_ISSUE_DATE>
+      <T_MAX_DATE>1287096765176</T_MAX_DATE>
+      <T_SEQUENCE_NUMBER>1</T_SEQUENCE_NUMBER>
+      <T_MASTER_KEY_ID>2</T_MASTER_KEY_ID>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>-1</OPCODE>
+    <DATA>
+    </DATA>
+  </RECORD>
+</EDITS>