Browse Source

HDFS-2187. Make EditLogInputStream act like an iterator over FSEditLogOps. Contributed by Ivan Kelly and Todd Lipcon.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1153996 13f79535-47bb-0310-9956-ffa450edef68
Todd Lipcon 13 years ago
parent
commit
53190cfa1d

+ 3 - 0
hdfs/CHANGES.txt

@@ -629,6 +629,9 @@ Trunk (unreleased changes)
     HDFS-2199. Move blockTokenSecretManager from FSNamesystem to BlockManager.
     (Uma Maheswara Rao G via szetszwo)
 
+    HDFS-2187. Make EditLogInputStream act like an iterator over FSEditLogOps
+    (Ivan Kelly and todd via todd)
+
   OPTIMIZATIONS
 
     HDFS-1458. Improve checkpoint performance by avoiding unnecessary image

+ 5 - 6
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/BackupImage.java

@@ -210,14 +210,13 @@ public class BackupImage extends FSImage {
       if (LOG.isTraceEnabled()) {
         LOG.debug("data:" + StringUtils.byteToHexString(data));
       }
-      backupInputStream.setBytes(data);
+
       FSEditLogLoader logLoader = new FSEditLogLoader(namesystem);
       int logVersion = storage.getLayoutVersion();
-      BufferedInputStream bin = new BufferedInputStream(backupInputStream);
-      DataInputStream in = new DataInputStream(bin);
-      Checksum checksum = FSEditLog.getChecksum();
-      int numLoaded = logLoader.loadEditRecords(logVersion, in, checksum, true,
-                                lastAppliedTxId + 1);
+      backupInputStream.setBytes(data, logVersion);
+
+      int numLoaded = logLoader.loadEditRecords(logVersion, backupInputStream, 
+                                                true, lastAppliedTxId + 1);
       if (numLoaded != numTxns) {
         throw new IOException("Batch of txns starting at txnid " +
             firstTxId + " was supposed to contain " + numTxns +

+ 25 - 14
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java

@@ -21,6 +21,8 @@ import java.io.DataInputStream;
 import java.io.ByteArrayInputStream;
 import java.io.IOException;
 
+import com.google.common.base.Preconditions;
+
 /**
  * An implementation of the abstract class {@link EditLogInputStream},
  * which is used to updates HDFS meta-data state on a backup node.
@@ -33,6 +35,9 @@ class EditLogBackupInputStream extends EditLogInputStream {
   String address; // sender address 
   private ByteBufferInputStream inner;
   private DataInputStream in;
+  private FSEditLogOp.Reader reader = null;
+  private FSEditLogLoader.PositionTrackingInputStream tracker = null;
+  private int version = 0;
 
   /**
    * A ByteArrayInputStream, which lets modify the underlying byte array.
@@ -60,7 +65,8 @@ class EditLogBackupInputStream extends EditLogInputStream {
   EditLogBackupInputStream(String name) throws IOException {
     address = name;
     inner = new ByteBufferInputStream();
-    in = new DataInputStream(inner);
+    in = null;
+    reader = null;
   }
 
   @Override // JournalStream
@@ -74,18 +80,20 @@ class EditLogBackupInputStream extends EditLogInputStream {
   }
 
   @Override
-  public int available() throws IOException {
-    return in.available();
+  public FSEditLogOp readOp() throws IOException {
+    Preconditions.checkState(reader != null,
+        "Must call setBytes() before readOp()");
+    return reader.readOp();
   }
 
   @Override
-  public int read() throws IOException {
-    return in.read();
+  public int getVersion() throws IOException {
+    return this.version;
   }
 
   @Override
-  public int read(byte[] b, int off, int len) throws IOException {
-    return in.read(b, off, len);
+  public long getPosition() {
+    return tracker.getPos();
   }
 
   @Override
@@ -99,16 +107,19 @@ class EditLogBackupInputStream extends EditLogInputStream {
     return inner.length();
   }
 
-  DataInputStream getDataInputStream() {
-    return in;
-  }
-
-  void setBytes(byte[] newBytes) throws IOException {
+  void setBytes(byte[] newBytes, int version) throws IOException {
     inner.setData(newBytes);
-    in.reset();
+    tracker = new FSEditLogLoader.PositionTrackingInputStream(inner);
+    in = new DataInputStream(tracker);
+
+    this.version = version;
+
+    reader = new FSEditLogOp.Reader(in, version);
   }
 
   void clear() throws IOException {
-    setBytes(null);
+    setBytes(null, 0);
+    reader = null;
+    this.version = 0;
   }
 }

+ 101 - 10
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java

@@ -21,18 +21,51 @@ package org.apache.hadoop.hdfs.server.namenode;
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.IOException;
+import java.io.BufferedInputStream;
+import java.io.EOFException;
+import java.io.DataInputStream;
+import org.apache.hadoop.hdfs.protocol.FSConstants;
+import org.apache.hadoop.hdfs.server.common.Storage;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogLoader.EditLogValidation;
+import org.apache.hadoop.io.IOUtils;
+
+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 {
-  private File file;
-  private FileInputStream fStream;
-
-  EditLogFileInputStream(File name) throws IOException {
+  private final File file;
+  private final FileInputStream fStream;
+  private final int logVersion;
+  private final FSEditLogOp.Reader reader;
+  private final FSEditLogLoader.PositionTrackingInputStream tracker;
+  
+  /**
+   * Open an EditLogInputStream for the given file.
+   * @param name filename to open
+   * @throws LogHeaderCorruptException if the header is either missing or
+   *         appears to be corrupt/truncated
+   * @throws IOException if an actual IO error occurs while reading the
+   *         header
+   */
+  EditLogFileInputStream(File name)
+      throws LogHeaderCorruptException, IOException {
     file = name;
     fStream = new FileInputStream(name);
+
+    BufferedInputStream bin = new BufferedInputStream(fStream);
+    tracker = new FSEditLogLoader.PositionTrackingInputStream(bin);
+    DataInputStream in = new DataInputStream(tracker);
+
+    try {
+      logVersion = readLogVersion(in);
+    } catch (EOFException eofe) {
+      throw new LogHeaderCorruptException("No header found in log");
+    }
+
+    reader = new FSEditLogOp.Reader(in, logVersion);
   }
 
   @Override // JournalStream
@@ -46,18 +79,18 @@ class EditLogFileInputStream extends EditLogInputStream {
   }
 
   @Override
-  public int available() throws IOException {
-    return fStream.available();
+  public FSEditLogOp readOp() throws IOException {
+    return reader.readOp();
   }
 
   @Override
-  public int read() throws IOException {
-    return fStream.read();
+  public int getVersion() throws IOException {
+    return logVersion;
   }
 
   @Override
-  public int read(byte[] b, int off, int len) throws IOException {
-    return fStream.read(b, off, len);
+  public long getPosition() {
+    return tracker.getPos();
   }
 
   @Override
@@ -76,4 +109,62 @@ class EditLogFileInputStream extends EditLogInputStream {
     return getName();
   }
 
+  static FSEditLogLoader.EditLogValidation validateEditLog(File file) throws IOException {
+    EditLogFileInputStream in;
+    try {
+      in = new EditLogFileInputStream(file);
+    } catch (LogHeaderCorruptException corrupt) {
+      // If it's missing its header, this is equivalent to no transactions
+      FSImage.LOG.warn("Log at " + file + " has no valid header",
+          corrupt);
+      return new FSEditLogLoader.EditLogValidation(0, 0);
+    }
+    
+    try {
+      return FSEditLogLoader.validateEditLog(in);
+    } finally {
+      IOUtils.closeStream(in);
+    }
+  }
+
+  /**
+   * Read the header of fsedit log
+   * @param in fsedit stream
+   * @return the edit log version number
+   * @throws IOException if error occurs
+   */
+  @VisibleForTesting
+  static int readLogVersion(DataInputStream in)
+      throws IOException, LogHeaderCorruptException {
+    int logVersion;
+    try {
+      logVersion = in.readInt();
+    } catch (EOFException eofe) {
+      throw new LogHeaderCorruptException(
+          "Reached EOF when reading log header");
+    }
+    if (logVersion < FSConstants.LAYOUT_VERSION) { // future version
+      throw new LogHeaderCorruptException(
+          "Unexpected version of the file system log file: "
+          + logVersion + ". Current version = "
+          + FSConstants.LAYOUT_VERSION + ".");
+    }
+    assert logVersion <= Storage.LAST_UPGRADABLE_LAYOUT_VERSION :
+      "Unsupported version " + logVersion;
+    return logVersion;
+  }
+  
+  /**
+   * Exception indicating that the header of an edits log file is
+   * corrupted. This can be because the header is not present,
+   * or because the header data is invalid (eg claims to be
+   * over a newer version than the running NameNode)
+   */
+  static class LogHeaderCorruptException extends IOException {
+    private static final long serialVersionUID = 1L;
+
+    private LogHeaderCorruptException(String msg) {
+      super(msg);
+    }
+  }
 }

+ 30 - 20
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java

@@ -17,10 +17,8 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import java.io.BufferedInputStream;
-import java.io.DataInputStream;
+import java.io.Closeable;
 import java.io.IOException;
-import java.io.InputStream;
 
 /**
  * A generic abstract class to support reading edits log data from 
@@ -29,29 +27,41 @@ import java.io.InputStream;
  * It should stream bytes from the storage exactly as they were written
  * into the #{@link EditLogOutputStream}.
  */
-abstract class EditLogInputStream extends InputStream
-implements JournalStream {
-  /** {@inheritDoc} */
-  public abstract int available() throws IOException;
-
-  /** {@inheritDoc} */
-  public abstract int read() throws IOException;
+abstract class EditLogInputStream implements JournalStream, Closeable {
+  /**
+   * Close the stream.
+   * @throws IOException if an error occurred while closing
+   */
+  public abstract void close() throws IOException;
 
-  /** {@inheritDoc} */
-  public abstract int read(byte[] b, int off, int len) throws IOException;
+  /** 
+   * Read an operation from the stream
+   * @return an operation from the stream or null if at end of stream
+   * @throws IOException if there is an error reading from the stream
+   */
+  public abstract FSEditLogOp readOp() throws IOException;
 
-  /** {@inheritDoc} */
-  public abstract void close() throws IOException;
+  /** 
+   * Get the layout version of the data in the stream.
+   * @return the layout version of the ops in the stream.
+   * @throws IOException if there is an error reading the version
+   */
+  public abstract int getVersion() throws IOException;
 
   /**
-   * Return the size of the current edits log.
+   * Get the "position" of in the stream. This is useful for 
+   * debugging and operational purposes.
+   *
+   * Different stream types can have a different meaning for 
+   * what the position is. For file streams it means the byte offset
+   * from the start of the file.
+   *
+   * @return the position in the stream
    */
-  abstract long length() throws IOException;
+  public abstract long getPosition();
 
   /**
-   * Return DataInputStream based on this edit stream.
+   * Return the size of the current edits log.
    */
-  DataInputStream getDataInputStream() {
-    return new DataInputStream(new BufferedInputStream(this));
-  }
+  abstract long length() throws IOException;
 }

+ 0 - 15
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java

@@ -20,8 +20,6 @@ package org.apache.hadoop.hdfs.server.namenode;
 import java.io.IOException;
 import java.util.Iterator;
 import java.util.List;
-import java.util.zip.Checksum;
-import java.util.zip.CheckedOutputStream;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -44,7 +42,6 @@ import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.security.token.delegation.DelegationKey;
-import org.apache.hadoop.util.PureJavaCrc32;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
@@ -116,18 +113,6 @@ public class FSEditLog  {
 
   private NNStorage storage;
 
-  private static ThreadLocal<Checksum> localChecksum =
-    new ThreadLocal<Checksum>() {
-    protected Checksum initialValue() {
-      return new PureJavaCrc32();
-    }
-  };
-
-  /** Get a thread local checksum */
-  public static Checksum getChecksum() {
-    return localChecksum.get();
-  }
-
   private static class TransactionId {
     public long txid;
 

+ 54 - 60
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java

@@ -37,7 +37,8 @@ import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.common.Storage;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.LogHeader;
+import org.apache.hadoop.hdfs.server.namenode.EditLogFileInputStream.LogHeaderCorruptException;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogLoader.EditLogValidation;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.Reader;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCloseOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.CancelDelegationTokenOp;
@@ -60,6 +61,7 @@ import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SymlinkOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.TimesOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateMasterKeyOp;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
+import org.apache.hadoop.io.IOUtils;
 
 public class FSEditLogLoader {
   private final FSNamesystem fsNamesys;
@@ -84,29 +86,25 @@ public class FSEditLogLoader {
   }
 
   int loadFSEdits(EditLogInputStream edits, boolean closeOnExit,
-      long expectedStartingTxId)
-  throws IOException {
-    BufferedInputStream bin = new BufferedInputStream(edits);
-    DataInputStream in = new DataInputStream(bin);
-
+                  long expectedStartingTxId)
+      throws IOException {
     int numEdits = 0;
+    int logVersion = edits.getVersion();
 
     try {
-      LogHeader header = LogHeader.read(in);
-      numEdits = loadEditRecords(
-          header.logVersion, in, header.checksum, false,
-          expectedStartingTxId);
+      numEdits = loadEditRecords(logVersion, edits, false, 
+                                 expectedStartingTxId);
     } finally {
-      if(closeOnExit)
-        in.close();
+      if(closeOnExit) {
+        edits.close();
+      }
     }
     
     return numEdits;
   }
 
   @SuppressWarnings("deprecation")
-  int loadEditRecords(int logVersion, DataInputStream in,
-                      Checksum checksum, boolean closeOnExit,
+  int loadEditRecords(int logVersion, EditLogInputStream in, boolean closeOnExit,
                       long expectedStartingTxId)
       throws IOException {
     FSDirectory fsDir = fsNamesys.dir;
@@ -123,10 +121,6 @@ public class FSEditLogLoader {
     fsNamesys.writeLock();
     fsDir.writeLock();
 
-    // Keep track of the file offsets of the last several opcodes.
-    // This is handy when manually recovering corrupted edits files.
-    PositionTrackingInputStream tracker = new PositionTrackingInputStream(in);
-    in = new DataInputStream(tracker);
     long recentOpcodeOffsets[] = new long[4];
     Arrays.fill(recentOpcodeOffsets, -1);
 
@@ -134,12 +128,10 @@ public class FSEditLogLoader {
       long txId = expectedStartingTxId - 1;
 
       try {
-        FSEditLogOp.Reader reader = new FSEditLogOp.Reader(in, logVersion,
-                                                           checksum);
         FSEditLogOp op;
-        while ((op = reader.readOp()) != null) {
+        while ((op = in.readOp()) != null) {
           recentOpcodeOffsets[numEdits % recentOpcodeOffsets.length] =
-              tracker.getPos();
+            in.getPosition();
           if (LayoutVersion.supports(Feature.STORED_TXIDS, logVersion)) {
             long thisTxId = op.txid;
             if (thisTxId != txId + 1) {
@@ -421,7 +413,7 @@ public class FSEditLogLoader {
       // Catch Throwable because in the case of a truly corrupt edits log, any
       // sort of error might be thrown (NumberFormat, NullPointer, EOF, etc.)
       StringBuilder sb = new StringBuilder();
-      sb.append("Error replaying edit log at offset " + tracker.getPos());
+      sb.append("Error replaying edit log at offset " + in.getPosition());
       if (recentOpcodeOffsets[0] != -1) {
         Arrays.sort(recentOpcodeOffsets);
         sb.append("\nRecent opcode offsets:");
@@ -480,49 +472,50 @@ public class FSEditLogLoader {
     }
   }
   
+  static EditLogValidation validateEditLog(File file) throws IOException {
+    EditLogFileInputStream in;
+    try {
+      in = new EditLogFileInputStream(file);
+    } catch (LogHeaderCorruptException corrupt) {
+      // If it's missing its header, this is equivalent to no transactions
+      FSImage.LOG.warn("Log at " + file + " has no valid header",
+          corrupt);
+      return new EditLogValidation(0, 0);
+    }
+    
+    try {
+      return validateEditLog(in);
+    } finally {
+      IOUtils.closeStream(in);
+    }
+  }
+
   /**
-   * Return the number of valid transactions in the file. If the file is
+   * Return the number of valid transactions in the stream. If the stream is
    * truncated during the header, returns a value indicating that there are
-   * 0 valid transactions.
-   * @throws IOException if the file cannot be read due to an IO error (eg
+   * 0 valid transactions. This reads through the stream but does not close
+   * it.
+   * @throws IOException if the stream cannot be read due to an IO error (eg
    *                     if the log does not exist)
    */
-  static EditLogValidation validateEditLog(File f) throws IOException {
-    FileInputStream fis = new FileInputStream(f);
+  static EditLogValidation validateEditLog(EditLogInputStream in) {
+    long numValid = 0;
+    long lastPos = 0;
     try {
-      PositionTrackingInputStream tracker = new PositionTrackingInputStream(
-          new BufferedInputStream(fis));
-      DataInputStream dis = new DataInputStream(tracker);
-      LogHeader header; 
-      try {
-        header = LogHeader.read(dis);
-      } catch (Throwable t) {
-        FSImage.LOG.debug("Unable to read header from " + f +
-            " -> no valid transactions in this file.");
-        return new EditLogValidation(0, 0);
-      }
-      
-      Reader reader = new FSEditLogOp.Reader(dis, header.logVersion, header.checksum);
-      long numValid = 0;
-      long lastPos = 0;
-      try {
-        while (true) {
-          lastPos = tracker.getPos();
-          if (reader.readOp() == null) {
-            break;
-          }
-          numValid++;
+      while (true) {
+        lastPos = in.getPosition();
+        if (in.readOp() == null) {
+          break;
         }
-      } catch (Throwable t) {
-        // Catch Throwable and not just IOE, since bad edits may generate
-        // NumberFormatExceptions, AssertionErrors, OutOfMemoryErrors, etc.
-        FSImage.LOG.debug("Caught exception after reading " + numValid +
-            " ops from " + f + " while determining its valid length.", t);
+        numValid++;
       }
-      return new EditLogValidation(lastPos, numValid);
-    } finally {
-      fis.close();
+    } catch (Throwable t) {
+      // Catch Throwable and not just IOE, since bad edits may generate
+      // NumberFormatExceptions, AssertionErrors, OutOfMemoryErrors, etc.
+      FSImage.LOG.debug("Caught exception after reading " + numValid +
+          " ops from " + in + " while determining its valid length.", t);
     }
+    return new EditLogValidation(lastPos, numValid);
   }
   
   static class EditLogValidation {
@@ -536,9 +529,9 @@ public class FSEditLogLoader {
   }
 
   /**
-   * Stream wrapper that keeps track of the current file position.
+   * Stream wrapper that keeps track of the current stream position.
    */
-  private static class PositionTrackingInputStream extends FilterInputStream {
+  static class PositionTrackingInputStream extends FilterInputStream {
     private long curPos = 0;
     private long markPos = -1;
 
@@ -582,4 +575,5 @@ public class FSEditLogLoader {
       return curPos;
     }
   }
+
 }

+ 13 - 64
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java

@@ -36,6 +36,7 @@ import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.Storage;
+import org.apache.hadoop.util.PureJavaCrc32;
 
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.*;
 import org.apache.hadoop.security.token.delegation.DelegationKey;
@@ -1323,71 +1324,17 @@ public abstract class FSEditLogOp {
       return longWritable.get();
     }
   }
-  
-  /**
-   * Class to encapsulate the header at the top of a log file.
-   */
-  static class LogHeader {
-    final int logVersion;
-    final Checksum checksum;
-
-    public LogHeader(int logVersion, Checksum checksum) {
-      this.logVersion = logVersion;
-      this.checksum = checksum;
-    }
-
-    static LogHeader read(DataInputStream in) throws IOException {
-      int logVersion = 0;
-
-      logVersion = FSEditLogOp.LogHeader.readLogVersion(in);
-      Checksum checksum = null;
-      if (LayoutVersion.supports(Feature.EDITS_CHESKUM, logVersion)) {
-        checksum = FSEditLog.getChecksum();
-      }
-      return new LogHeader(logVersion, checksum);
-    }
-    
-    /**
-     * Read the header of fsedit log
-     * @param in fsedit stream
-     * @return the edit log version number
-     * @throws IOException if error occurs
-     */
-    private static int readLogVersion(DataInputStream in) throws IOException {
-      int logVersion = 0;
-      // Read log file version. Could be missing.
-      in.mark(4);
-      // If edits log is greater than 2G, available method will return negative
-      // numbers, so we avoid having to call available
-      boolean available = true;
-      try {
-        logVersion = in.readByte();
-      } catch (EOFException e) {
-        available = false;
-      }
-      if (available) {
-        in.reset();
-        logVersion = in.readInt();
-        if (logVersion < FSConstants.LAYOUT_VERSION) // future version
-          throw new IOException(
-              "Unexpected version of the file system log file: "
-              + logVersion + ". Current version = "
-              + FSConstants.LAYOUT_VERSION + ".");
-      }
-      assert logVersion <= Storage.LAST_UPGRADABLE_LAYOUT_VERSION :
-        "Unsupported version " + logVersion;
-      return logVersion;
-    }
-  }
 
   /**
    * Class for writing editlog ops
    */
   public static class Writer {
     private final DataOutputBuffer buf;
+    private final Checksum checksum;
 
     public Writer(DataOutputBuffer out) {
       this.buf = out;
+      this.checksum = new PureJavaCrc32();
     }
 
     /**
@@ -1402,7 +1349,6 @@ public abstract class FSEditLogOp {
       buf.writeLong(op.txid);
       op.writeFields(buf);
       int end = buf.getLength();
-      Checksum checksum = FSEditLog.getChecksum();
       checksum.reset();
       checksum.update(buf.getData(), start, end-start);
       int sum = (int)checksum.getValue();
@@ -1422,19 +1368,22 @@ public abstract class FSEditLogOp {
      * Construct the reader
      * @param in The stream to read from.
      * @param logVersion The version of the data coming from the stream.
-     * @param checksum Checksum being used with input stream.
      */
     @SuppressWarnings("deprecation")
-    public Reader(DataInputStream in, int logVersion,
-                  Checksum checksum) {
-      if (checksum != null) {
+    public Reader(DataInputStream in, int logVersion) {
+      this.logVersion = logVersion;
+      if (LayoutVersion.supports(Feature.EDITS_CHESKUM, logVersion)) {
+        this.checksum = new PureJavaCrc32();
+      } else {
+        this.checksum = null;
+      }
+
+      if (this.checksum != null) {
         this.in = new DataInputStream(
-            new CheckedInputStream(in, checksum));
+            new CheckedInputStream(in, this.checksum));
       } else {
         this.in = in;
       }
-      this.logVersion = logVersion;
-      this.checksum = checksum;
     }
 
     /**

+ 1 - 1
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSImageTransactionalStorageInspector.java

@@ -588,7 +588,7 @@ class FSImageTransactionalStorageInspector extends FSImageStorageInspector {
 
     EditLogValidation validateLog() throws IOException {
       if (cachedValidation == null) {
-        cachedValidation = FSEditLogLoader.validateEditLog(file);
+        cachedValidation = EditLogFileInputStream.validateEditLog(file);
       }
       return cachedValidation;
     }

+ 27 - 11
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java

@@ -676,28 +676,44 @@ public class TestEditLog extends TestCase {
   private static class EditLogByteInputStream extends EditLogInputStream {
     private InputStream input;
     private long len;
+    private int version;
+    private FSEditLogOp.Reader reader = null;
+    private FSEditLogLoader.PositionTrackingInputStream tracker = null;
 
-    public EditLogByteInputStream(byte[] data) {
+    public EditLogByteInputStream(byte[] data) throws IOException {
       len = data.length;
       input = new ByteArrayInputStream(data);
-    }
 
-    public int available() throws IOException {
-      return input.available();
-    }
-    
-    public int read() throws IOException {
-      return input.read();
+      BufferedInputStream bin = new BufferedInputStream(input);
+      DataInputStream in = new DataInputStream(bin);
+      version = EditLogFileInputStream.readLogVersion(in);
+      tracker = new FSEditLogLoader.PositionTrackingInputStream(in);
+      in = new DataInputStream(tracker);
+            
+      reader = new FSEditLogOp.Reader(in, version);
     }
     
+    @Override
     public long length() throws IOException {
       return len;
     }
-    
-    public int read(byte[] b, int off, int len) throws IOException {
-      return input.read(b, off, len);
+  
+    @Override
+    public long getPosition() {
+      return tracker.getPos();
+    }
+
+    @Override
+    public FSEditLogOp readOp() throws IOException {
+      return reader.readOp();
+    }
+
+    @Override
+    public int getVersion() throws IOException {
+      return version;
     }
 
+    @Override
     public void close() throws IOException {
       input.close();
     }

+ 2 - 2
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileOutputStream.java

@@ -61,7 +61,7 @@ public class TestEditLogFileOutputStream {
       .getStorage().getStorageDir(0);
     File editLog = NNStorage.getInProgressEditsFile(sd, 1);
 
-    EditLogValidation validation = FSEditLogLoader.validateEditLog(editLog);
+    EditLogValidation validation = EditLogFileInputStream.validateEditLog(editLog);
     assertEquals("Edit log should contain a header as valid length",
         HEADER_LEN, validation.validLength);
     assertEquals(1, validation.numTransactions);
@@ -73,7 +73,7 @@ public class TestEditLogFileOutputStream {
         new FsPermission((short)777));
 
     long oldLength = validation.validLength;
-    validation = FSEditLogLoader.validateEditLog(editLog);
+    validation = EditLogFileInputStream.validateEditLog(editLog);
     assertTrue("Edit log should have more valid data after writing a txn " +
         "(was: " + oldLength + " now: " + validation.validLength + ")",
         validation.validLength > oldLength);

+ 6 - 6
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java

@@ -186,7 +186,7 @@ public class TestFSEditLogLoader {
 
     // Make sure that uncorrupted log has the expected length and number
     // of transactions.
-    EditLogValidation validation = FSEditLogLoader.validateEditLog(logFile);
+    EditLogValidation validation = EditLogFileInputStream.validateEditLog(logFile);
     assertEquals(NUM_TXNS + 2, validation.numTransactions);
     assertEquals(validLength, validation.validLength);
     
@@ -202,7 +202,7 @@ public class TestFSEditLogLoader {
       // Restore backup, truncate the file exactly before the txn
       Files.copy(logFileBak, logFile);
       truncateFile(logFile, txOffset);
-      validation = FSEditLogLoader.validateEditLog(logFile);
+      validation = EditLogFileInputStream.validateEditLog(logFile);
       assertEquals("Failed when truncating to length " + txOffset,
           txid - 1, validation.numTransactions);
       assertEquals(txOffset, validation.validLength);
@@ -211,7 +211,7 @@ public class TestFSEditLogLoader {
       // also isn't valid
       Files.copy(logFileBak, logFile);
       truncateFile(logFile, txOffset + 1);
-      validation = FSEditLogLoader.validateEditLog(logFile);
+      validation = EditLogFileInputStream.validateEditLog(logFile);
       assertEquals("Failed when truncating to length " + (txOffset + 1),
           txid - 1, validation.numTransactions);
       assertEquals(txOffset, validation.validLength);
@@ -219,7 +219,7 @@ public class TestFSEditLogLoader {
       // Restore backup, corrupt the txn opcode
       Files.copy(logFileBak, logFile);
       corruptByteInFile(logFile, txOffset);
-      validation = FSEditLogLoader.validateEditLog(logFile);
+      validation = EditLogFileInputStream.validateEditLog(logFile);
       assertEquals("Failed when corrupting txn opcode at " + txOffset,
           txid - 1, validation.numTransactions);
       assertEquals(txOffset, validation.validLength);
@@ -227,7 +227,7 @@ public class TestFSEditLogLoader {
       // Restore backup, corrupt a byte a few bytes into the txn
       Files.copy(logFileBak, logFile);
       corruptByteInFile(logFile, txOffset+5);
-      validation = FSEditLogLoader.validateEditLog(logFile);
+      validation = EditLogFileInputStream.validateEditLog(logFile);
       assertEquals("Failed when corrupting txn data at " + (txOffset+5),
           txid - 1, validation.numTransactions);
       assertEquals(txOffset, validation.validLength);
@@ -240,7 +240,7 @@ public class TestFSEditLogLoader {
     for (long offset = 0; offset < validLength; offset++) {
       Files.copy(logFileBak, logFile);
       corruptByteInFile(logFile, offset);
-      EditLogValidation val = FSEditLogLoader.validateEditLog(logFile);
+      EditLogValidation val = EditLogFileInputStream.validateEditLog(logFile);
       assertTrue(val.numTransactions >= prevNumValid);
       prevNumValid = val.numTransactions;
     }