|
@@ -24,12 +24,9 @@ import java.io.IOException;
|
|
|
import java.io.RandomAccessFile;
|
|
|
import java.nio.ByteBuffer;
|
|
|
import java.nio.channels.FileChannel;
|
|
|
-import java.util.zip.Checksum;
|
|
|
|
|
|
import org.apache.hadoop.hdfs.protocol.FSConstants;
|
|
|
-import org.apache.hadoop.io.DataOutputBuffer;
|
|
|
import org.apache.hadoop.io.IOUtils;
|
|
|
-import org.apache.hadoop.io.Writable;
|
|
|
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
|
|
|
@@ -43,10 +40,7 @@ class EditLogFileOutputStream extends EditLogOutputStream {
|
|
|
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
|
|
|
- private FSEditLogOp.Writer writer;
|
|
|
- final private int initBufferSize; // inital buffer size
|
|
|
+ private EditsDoubleBuffer doubleBuf;
|
|
|
static ByteBuffer fill = ByteBuffer.allocateDirect(1024 * 1024); // preallocation, 1MB
|
|
|
|
|
|
static {
|
|
@@ -68,10 +62,7 @@ class EditLogFileOutputStream extends EditLogOutputStream {
|
|
|
EditLogFileOutputStream(File name, int size) throws IOException {
|
|
|
super();
|
|
|
file = name;
|
|
|
- initBufferSize = size;
|
|
|
- bufCurrent = new DataOutputBuffer(size);
|
|
|
- bufReady = new DataOutputBuffer(size);
|
|
|
- writer = new FSEditLogOp.Writer(bufCurrent);
|
|
|
+ doubleBuf = new EditsDoubleBuffer(size);
|
|
|
RandomAccessFile rp = new RandomAccessFile(name, "rw");
|
|
|
fp = new FileOutputStream(rp.getFD()); // open for append
|
|
|
fc = rp.getChannel();
|
|
@@ -91,23 +82,13 @@ class EditLogFileOutputStream extends EditLogOutputStream {
|
|
|
/** {@inheritDoc} */
|
|
|
@Override
|
|
|
void write(FSEditLogOp op) throws IOException {
|
|
|
- int start = bufCurrent.getLength();
|
|
|
-
|
|
|
- writer.writeOp(op);
|
|
|
-
|
|
|
- // write transaction checksum
|
|
|
- int end = bufCurrent.getLength();
|
|
|
- Checksum checksum = FSEditLog.getChecksum();
|
|
|
- checksum.reset();
|
|
|
- checksum.update(bufCurrent.getData(), start, end-start);
|
|
|
- int sum = (int)checksum.getValue();
|
|
|
- bufCurrent.writeInt(sum);
|
|
|
+ doubleBuf.writeOp(op);
|
|
|
}
|
|
|
|
|
|
/** {@inheritDoc} */
|
|
|
@Override
|
|
|
void writeRaw(byte[] bytes, int offset, int length) throws IOException {
|
|
|
- bufCurrent.write(bytes, offset, length);
|
|
|
+ doubleBuf.writeRaw(bytes, offset, length);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -117,7 +98,7 @@ class EditLogFileOutputStream extends EditLogOutputStream {
|
|
|
void create() throws IOException {
|
|
|
fc.truncate(0);
|
|
|
fc.position(0);
|
|
|
- bufCurrent.writeInt(FSConstants.LAYOUT_VERSION);
|
|
|
+ doubleBuf.getCurrentBuf().writeInt(FSConstants.LAYOUT_VERSION);
|
|
|
setReadyToFlush();
|
|
|
flush();
|
|
|
}
|
|
@@ -128,23 +109,11 @@ class EditLogFileOutputStream extends EditLogOutputStream {
|
|
|
// close should have been called after all pending transactions
|
|
|
// have been flushed & synced.
|
|
|
// if already closed, just skip
|
|
|
- if(bufCurrent != null)
|
|
|
- {
|
|
|
- int bufSize = bufCurrent.size();
|
|
|
- if (bufSize != 0) {
|
|
|
- throw new IOException("FSEditStream has " + bufSize
|
|
|
- + " bytes still to be flushed and cannot " + "be closed.");
|
|
|
- }
|
|
|
- bufCurrent.close();
|
|
|
- bufCurrent = null;
|
|
|
- writer = null;
|
|
|
- }
|
|
|
-
|
|
|
- if(bufReady != null) {
|
|
|
- bufReady.close();
|
|
|
- bufReady = null;
|
|
|
+ if (doubleBuf != null) {
|
|
|
+ doubleBuf.close();
|
|
|
+ doubleBuf = null;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
// remove the last INVALID marker from transaction log.
|
|
|
if (fc != null && fc.isOpen()) {
|
|
|
fc.truncate(fc.position());
|
|
@@ -156,9 +125,8 @@ class EditLogFileOutputStream extends EditLogOutputStream {
|
|
|
fp = null;
|
|
|
}
|
|
|
} finally {
|
|
|
- IOUtils.cleanup(FSNamesystem.LOG, bufCurrent, bufReady, fc, fp);
|
|
|
- bufCurrent = bufReady = null;
|
|
|
- writer = null;
|
|
|
+ IOUtils.cleanup(FSNamesystem.LOG, fc, fp);
|
|
|
+ doubleBuf = null;
|
|
|
fc = null;
|
|
|
fp = null;
|
|
|
}
|
|
@@ -170,12 +138,8 @@ class EditLogFileOutputStream extends EditLogOutputStream {
|
|
|
*/
|
|
|
@Override
|
|
|
void setReadyToFlush() throws IOException {
|
|
|
- assert bufReady.size() == 0 : "previous data is not flushed yet";
|
|
|
- bufCurrent.write(FSEditLogOpCodes.OP_INVALID.getOpCode()); // insert eof marker
|
|
|
- DataOutputBuffer tmp = bufReady;
|
|
|
- bufReady = bufCurrent;
|
|
|
- bufCurrent = tmp;
|
|
|
- writer = new FSEditLogOp.Writer(bufCurrent);
|
|
|
+ doubleBuf.getCurrentBuf().write(FSEditLogOpCodes.OP_INVALID.getOpCode()); // insert eof marker
|
|
|
+ doubleBuf.setReadyToFlush();
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -185,8 +149,7 @@ class EditLogFileOutputStream extends EditLogOutputStream {
|
|
|
@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
|
|
|
+ doubleBuf.flushTo(fp);
|
|
|
fc.force(false); // metadata updates not needed because of preallocation
|
|
|
fc.position(fc.position() - 1); // skip back the end-of-file marker
|
|
|
}
|
|
@@ -196,7 +159,7 @@ class EditLogFileOutputStream extends EditLogOutputStream {
|
|
|
*/
|
|
|
@Override
|
|
|
public boolean shouldForceSync() {
|
|
|
- return bufReady.size() >= initBufferSize;
|
|
|
+ return doubleBuf.shouldForceSync();
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -205,8 +168,8 @@ class EditLogFileOutputStream extends EditLogOutputStream {
|
|
|
@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();
|
|
|
+ return fc.size() - EDITS_FILE_HEADER_SIZE_BYTES +
|
|
|
+ doubleBuf.countBufferedBytes();
|
|
|
}
|
|
|
|
|
|
// allocate a big chunk of data
|