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