You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by sh...@apache.org on 2009/05/15 02:32:11 UTC

svn commit: r774970 - in /hadoop/core/trunk: ./ src/hdfs/org/apache/hadoop/hdfs/server/namenode/ src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/

Author: shv
Date: Fri May 15 00:32:11 2009
New Revision: 774970

URL: http://svn.apache.org/viewvc?rev=774970&view=rev
Log:
HADOOP-5721. Factor out EditLogFileInputStream and EditLogFileOutputStream into independent classes. Contributed by Luca Telloli & Flavio Junqueira.

Added:
    hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java
    hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java
Modified:
    hadoop/core/trunk/CHANGES.txt
    hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/BackupStorage.java
    hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
    hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSImage.java
    hadoop/core/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java

Modified: hadoop/core/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/CHANGES.txt?rev=774970&r1=774969&r2=774970&view=diff
==============================================================================
--- hadoop/core/trunk/CHANGES.txt (original)
+++ hadoop/core/trunk/CHANGES.txt Fri May 15 00:32:11 2009
@@ -353,6 +353,9 @@
     HADOOP-5824. Deprecate DataTransferProtocol.OP_READ_METADATA and remove
     the corresponding unused codes.  (Kan Zhang via szetszwo)
 
+    HADOOP-5721. Factor out EditLogFileInputStream and EditLogFileOutputStream
+    into independent classes. (Luca Telloli & Flavio Junqueira via shv)
+
   OPTIMIZATIONS
 
     HADOOP-5595. NameNode does not need to run a replicator to choose a

Modified: hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/BackupStorage.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/BackupStorage.java?rev=774970&r1=774969&r2=774970&view=diff
==============================================================================
--- hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/BackupStorage.java (original)
+++ hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/BackupStorage.java Fri May 15 00:32:11 2009
@@ -26,7 +26,7 @@
 import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
 import org.apache.hadoop.hdfs.server.namenode.FSImage;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLog.EditLogFileInputStream;
+import org.apache.hadoop.hdfs.server.namenode.EditLogFileInputStream;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.io.LongWritable;

Added: hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java?rev=774970&view=auto
==============================================================================
--- hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java (added)
+++ hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java Fri May 15 00:32:11 2009
@@ -0,0 +1,73 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.server.namenode;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+
+/**
+ * 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 {
+    file = name;
+    fStream = new FileInputStream(name);
+  }
+
+  @Override // JournalStream
+  public String getName() {
+    return file.getPath();
+  }
+
+  @Override // JournalStream
+  public JournalType getType() {
+    return JournalType.FILE;
+  }
+
+  @Override
+  public int available() throws IOException {
+    return fStream.available();
+  }
+
+  @Override
+  public int read() throws IOException {
+    return fStream.read();
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    return fStream.read(b, off, len);
+  }
+
+  @Override
+  public void close() throws IOException {
+    fStream.close();
+  }
+
+  @Override
+  long length() throws IOException {
+    // file size + size of both buffers
+    return file.length();
+  }
+}

Added: hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java?rev=774970&view=auto
==============================================================================
--- hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java (added)
+++ hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java Fri May 15 00:32:11 2009
@@ -0,0 +1,187 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.server.namenode;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+
+import org.apache.hadoop.hdfs.protocol.FSConstants;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * An implementation of the abstract class {@link EditLogOutputStream}, which
+ * stores edits in a local file.
+ */
+class EditLogFileOutputStream extends EditLogOutputStream {
+  private static int EDITS_FILE_HEADER_SIZE_BYTES = Integer.SIZE / Byte.SIZE;
+
+  private File file;
+  private FileOutputStream fp; // file stream for storing edit logs
+  private FileChannel fc; // channel of the file stream for sync
+  private DataOutputBuffer bufCurrent; // current buffer for writing
+  private DataOutputBuffer bufReady; // buffer ready for flushing
+  static ByteBuffer fill = ByteBuffer.allocateDirect(512); // preallocation
+
+  /**
+   * Creates output buffers and file object.
+   * 
+   * @param name
+   *          File name to store edit log
+   * @param size
+   *          Size of flush buffer
+   * @throws IOException
+   */
+  EditLogFileOutputStream(File name, int size) throws IOException {
+    super();
+    file = name;
+    bufCurrent = new DataOutputBuffer(size);
+    bufReady = new DataOutputBuffer(size);
+    RandomAccessFile rp = new RandomAccessFile(name, "rw");
+    fp = new FileOutputStream(rp.getFD()); // open for append
+    fc = rp.getChannel();
+    fc.position(fc.size());
+  }
+
+  @Override // JournalStream
+  public String getName() {
+    return file.getPath();
+  }
+
+  @Override // JournalStream
+  public JournalType getType() {
+    return JournalType.FILE;
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public void write(int b) throws IOException {
+    bufCurrent.write(b);
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  void write(byte op, Writable... writables) throws IOException {
+    write(op);
+    for (Writable w : writables) {
+      w.write(bufCurrent);
+    }
+  }
+
+  /**
+   * Create empty edits logs file.
+   */
+  @Override
+  void create() throws IOException {
+    fc.truncate(0);
+    fc.position(0);
+    bufCurrent.writeInt(FSConstants.LAYOUT_VERSION);
+    setReadyToFlush();
+    flush();
+  }
+
+  @Override
+  public void close() throws IOException {
+    // close should have been called after all pending transactions
+    // have been flushed & synced.
+    int bufSize = bufCurrent.size();
+    if (bufSize != 0) {
+      throw new IOException("FSEditStream has " + bufSize
+          + " bytes still to be flushed and cannot " + "be closed.");
+    }
+    bufCurrent.close();
+    bufReady.close();
+
+    // remove the last INVALID marker from transaction log.
+    fc.truncate(fc.position());
+    fp.close();
+
+    bufCurrent = bufReady = null;
+  }
+
+  /**
+   * All data that has been written to the stream so far will be flushed. New
+   * data can be still written to the stream while flushing is performed.
+   */
+  @Override
+  void setReadyToFlush() throws IOException {
+    assert bufReady.size() == 0 : "previous data is not flushed yet";
+    write(FSEditLog.OP_INVALID); // insert end-of-file marker
+    DataOutputBuffer tmp = bufReady;
+    bufReady = bufCurrent;
+    bufCurrent = tmp;
+  }
+
+  /**
+   * Flush ready buffer to persistent store. currentBuffer is not flushed as it
+   * accumulates new log records while readyBuffer will be flushed and synced.
+   */
+  @Override
+  protected void flushAndSync() throws IOException {
+    preallocate(); // preallocate file if necessary
+    bufReady.writeTo(fp); // write data to file
+    bufReady.reset(); // erase all data in the buffer
+    fc.force(false); // metadata updates not needed because of preallocation
+    fc.position(fc.position() - 1); // skip back the end-of-file marker
+  }
+
+  /**
+   * Return the size of the current edit log including buffered data.
+   */
+  @Override
+  long length() throws IOException {
+    // file size - header size + size of both buffers
+    return fc.size() - EDITS_FILE_HEADER_SIZE_BYTES + bufReady.size()
+        + bufCurrent.size();
+  }
+
+  // allocate a big chunk of data
+  private void preallocate() throws IOException {
+    long position = fc.position();
+    if (position + 4096 >= fc.size()) {
+      FSNamesystem.LOG.debug("Preallocating Edit log, current size "
+          + fc.size());
+      long newsize = position + 1024 * 1024; // 1MB
+      fill.position(0);
+      int written = fc.write(fill, newsize);
+      FSNamesystem.LOG.debug("Edit log size is now " + fc.size() + " written "
+          + written + " bytes " + " at offset " + newsize);
+    }
+  }
+
+  /**
+   * Operations like OP_JSPOOL_START and OP_CHECKPOINT_TIME should not be
+   * written into edits file.
+   */
+  @Override
+  boolean isOperationSupported(byte op) {
+    return op < FSEditLog.OP_JSPOOL_START - 1;
+  }
+
+  /**
+   * Returns the file associated with this stream.
+   */
+  File getFile() {
+    return file;
+  }
+}

Modified: hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java?rev=774970&r1=774969&r2=774970&view=diff
==============================================================================
--- hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java (original)
+++ hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java Fri May 15 00:32:11 2009
@@ -22,12 +22,7 @@
 import java.io.DataOutput;
 import java.io.EOFException;
 import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
 import java.io.IOException;
-import java.io.RandomAccessFile;
-import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
 import java.util.ArrayList;
 import java.util.Iterator;
 
@@ -47,7 +42,6 @@
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
 import org.apache.hadoop.io.ArrayWritable;
-import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.DeprecatedUTF8;
 import org.apache.hadoop.io.Writable;
@@ -91,7 +85,7 @@
   static final String NO_JOURNAL_STREAMS_WARNING = "!!! WARNING !!!" +
   		" File system changes are not persistent. No journal streams.";
 
-  private static int sizeFlushBuffer = 512*1024;
+  private volatile int sizeOutputFlushBuffer = 512*1024;
 
   private ArrayList<EditLogOutputStream> editStreams = null;
   private FSImage fsimage = null;
@@ -129,207 +123,6 @@
     }
   };
 
-  /**
-   * An implementation of the abstract class {@link EditLogOutputStream},
-   * which stores edits in a local file.
-   */
-  static private class EditLogFileOutputStream extends EditLogOutputStream {
-    private static int EDITS_FILE_HEADER_SIZE_BYTES = Integer.SIZE/Byte.SIZE;
-
-    private File file;
-    private FileOutputStream fp;    // file stream for storing edit logs 
-    private FileChannel fc;         // channel of the file stream for sync
-    private DataOutputBuffer bufCurrent;  // current buffer for writing
-    private DataOutputBuffer bufReady;    // buffer ready for flushing
-    static ByteBuffer fill = ByteBuffer.allocateDirect(512); // preallocation
-
-    EditLogFileOutputStream(File name) throws IOException {
-      super();
-      file = name;
-      bufCurrent = new DataOutputBuffer(sizeFlushBuffer);
-      bufReady = new DataOutputBuffer(sizeFlushBuffer);
-      RandomAccessFile rp = new RandomAccessFile(name, "rw");
-      fp = new FileOutputStream(rp.getFD()); // open for append
-      fc = rp.getChannel();
-      fc.position(fc.size());
-    }
-
-    @Override // JournalStream
-    public String getName() {
-      return file.getPath();
-    }
-
-    @Override // JournalStream
-    public JournalType getType() {
-      return JournalType.FILE;
-    }
-
-    /** {@inheritDoc} */
-    @Override
-    public void write(int b) throws IOException {
-      bufCurrent.write(b);
-    }
-
-    /** {@inheritDoc} */
-    @Override
-    void write(byte op, Writable ... writables) throws IOException {
-      write(op);
-      for(Writable w : writables) {
-        w.write(bufCurrent);
-      }
-    }
-
-    /**
-     * Create empty edits logs file.
-     */
-    @Override
-    void create() throws IOException {
-      fc.truncate(0);
-      fc.position(0);
-      bufCurrent.writeInt(FSConstants.LAYOUT_VERSION);
-      setReadyToFlush();
-      flush();
-    }
-
-    @Override
-    public void close() throws IOException {
-      // close should have been called after all pending transactions 
-      // have been flushed & synced.
-      int bufSize = bufCurrent.size();
-      if (bufSize != 0) {
-        throw new IOException("FSEditStream has " + bufSize +
-                              " bytes still to be flushed and cannot " +
-                              "be closed.");
-      } 
-      bufCurrent.close();
-      bufReady.close();
-
-      // remove the last INVALID marker from transaction log.
-      fc.truncate(fc.position());
-      fp.close();
-      
-      bufCurrent = bufReady = null;
-    }
-
-    /**
-     * All data that has been written to the stream so far will be flushed.
-     * New data can be still written to the stream while flushing is performed.
-     */
-    @Override
-    void setReadyToFlush() throws IOException {
-      assert bufReady.size() == 0 : "previous data is not flushed yet";
-      write(OP_INVALID);           // insert end-of-file marker
-      DataOutputBuffer tmp = bufReady;
-      bufReady = bufCurrent;
-      bufCurrent = tmp;
-    }
-
-    /**
-     * Flush ready buffer to persistent store.
-     * currentBuffer is not flushed as it accumulates new log records
-     * while readyBuffer will be flushed and synced.
-     */
-    @Override
-    protected void flushAndSync() throws IOException {
-      preallocate();            // preallocate file if necessary
-      bufReady.writeTo(fp);     // write data to file
-      bufReady.reset();         // erase all data in the buffer
-      fc.force(false);          // metadata updates not needed because of preallocation
-      fc.position(fc.position()-1); // skip back the end-of-file marker
-    }
-
-    /**
-     * Return the size of the current edit log including buffered data.
-     */
-    @Override
-    long length() throws IOException {
-      // file size - header size + size of both buffers
-      return fc.size() - EDITS_FILE_HEADER_SIZE_BYTES
-                       + bufReady.size() + bufCurrent.size();
-    }
-
-    // allocate a big chunk of data
-    private void preallocate() throws IOException {
-      long position = fc.position();
-      if (position + 4096 >= fc.size()) {
-        FSNamesystem.LOG.debug("Preallocating Edit log, current size " +
-                                fc.size());
-        long newsize = position + 1024*1024; // 1MB
-        fill.position(0);
-        int written = fc.write(fill, newsize);
-        FSNamesystem.LOG.debug("Edit log size is now " + fc.size() +
-                              " written " + written + " bytes " +
-                              " at offset " +  newsize);
-      }
-    }
-
-    /**
-     * Operations like OP_JSPOOL_START and OP_CHECKPOINT_TIME
-     * should not be written into edits file.
-     */
-    @Override
-    boolean isOperationSupported(byte op) {
-      return op < OP_JSPOOL_START - 1;
-    }
-
-    /**
-     * Returns the file associated with this stream
-     */
-    File getFile() {
-      return file;
-    }
-  }
-
-  /**
-   * An implementation of the abstract class {@link EditLogInputStream},
-   * which reads edits from a local file.
-   */
-  static class EditLogFileInputStream extends EditLogInputStream {
-    private File file;
-    private FileInputStream fStream;
-
-    EditLogFileInputStream(File name) throws IOException {
-      file = name;
-      fStream = new FileInputStream(name);
-    }
-
-    @Override // JournalStream
-    public String getName() {
-      return file.getPath();
-    }
-
-    @Override // JournalStream
-    public JournalType getType() {
-      return JournalType.FILE;
-    }
-
-    @Override
-    public int available() throws IOException {
-      return fStream.available();
-    }
-
-    @Override
-    public int read() throws IOException {
-      return fStream.read();
-    }
-
-    @Override
-    public int read(byte[] b, int off, int len) throws IOException {
-      return fStream.read(b, off, len);
-    }
-
-    @Override
-    public void close() throws IOException {
-      fStream.close();
-    }
-
-    @Override
-    long length() throws IOException {
-      // file size + size of both buffers
-      return file.length();
-    }
-  }
-
   FSEditLog(FSImage image) {
     fsimage = image;
     isSyncRunning = false;
@@ -363,7 +156,7 @@
    * 
    * @throws IOException
    */
-  public synchronized void open() throws IOException {
+  synchronized void open() throws IOException {
     numTransactions = totalTimeTransactions = numTransactionsBatchedInSync = 0;
     if (editStreams == null)
       editStreams = new ArrayList<EditLogOutputStream>();
@@ -388,13 +181,15 @@
   }
   
   
-  public synchronized void addNewEditLogStream(File eFile) throws IOException {
-    EditLogOutputStream eStream = new EditLogFileOutputStream(eFile);
+  synchronized void addNewEditLogStream(File eFile) throws IOException {
+    EditLogOutputStream eStream = new EditLogFileOutputStream(eFile,
+        sizeOutputFlushBuffer);
     editStreams.add(eStream);
   }
 
-  public synchronized void createEditLogFile(File name) throws IOException {
-    EditLogOutputStream eStream = new EditLogFileOutputStream(name);
+  synchronized void createEditLogFile(File name) throws IOException {
+    EditLogOutputStream eStream = new EditLogFileOutputStream(name,
+        sizeOutputFlushBuffer);
     eStream.create();
     eStream.close();
   }
@@ -1279,7 +1074,7 @@
    * @param dest new stream path relative to the storage directory root.
    * @throws IOException
    */
-  void divertFileStreams(String dest) throws IOException {
+  synchronized void divertFileStreams(String dest) throws IOException {
     assert getNumEditStreams() >= getNumEditsDirs() :
       "Inconsistent number of streams";
     ArrayList<EditLogOutputStream> errorStreams = null;
@@ -1296,7 +1091,8 @@
         // close old stream
         closeStream(eStream);
         // create new stream
-        eStream = new EditLogFileOutputStream(new File(sd.getRoot(), dest));
+        eStream = new EditLogFileOutputStream(new File(sd.getRoot(), dest),
+            sizeOutputFlushBuffer);
         eStream.create();
         // replace by the new stream
         itE.replace(eStream);
@@ -1357,7 +1153,7 @@
           }
         }
         // open new stream
-        eStream = new EditLogFileOutputStream(editFile);
+        eStream = new EditLogFileOutputStream(editFile, sizeOutputFlushBuffer);
         // replace by the new stream
         itE.replace(eStream);
       } catch (IOException e) {
@@ -1394,8 +1190,8 @@
   }
 
   // sets the initial capacity of the flush buffer.
-  static void setBufferCapacity(int size) {
-    sizeFlushBuffer = size;
+  public void setBufferCapacity(int size) {
+    sizeOutputFlushBuffer = size;
   }
 
   /**

Modified: hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSImage.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSImage.java?rev=774970&r1=774969&r2=774970&view=diff
==============================================================================
--- hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSImage.java (original)
+++ hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSImage.java Fri May 15 00:32:11 2009
@@ -55,7 +55,7 @@
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.BlocksMap.BlockInfo;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLog.EditLogFileInputStream;
+import org.apache.hadoop.hdfs.server.namenode.EditLogFileInputStream;
 import org.apache.hadoop.hdfs.server.protocol.CheckpointCommand;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;

Modified: hadoop/core/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java?rev=774970&r1=774969&r2=774970&view=diff
==============================================================================
--- hadoop/core/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java (original)
+++ hadoop/core/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java Fri May 15 00:32:11 2009
@@ -25,7 +25,7 @@
 import org.apache.hadoop.fs.permission.*;
 
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLog.EditLogFileInputStream;
+import org.apache.hadoop.hdfs.server.namenode.EditLogFileInputStream;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSImage.NameNodeDirType;
 import org.apache.hadoop.hdfs.server.namenode.FSImage.NameNodeFile;
@@ -101,7 +101,7 @@
       FSEditLog editLog = fsimage.getEditLog();
   
       // set small size of flush buffer
-      FSEditLog.setBufferCapacity(2048);
+      editLog.setBufferCapacity(2048);
       editLog.close();
       editLog.open();