mirror of https://github.com/apache/lucene.git
LUCENE-2453: Make Index Output Buffer Size Configurable
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1437295 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
6ba9d0b641
commit
384e45b699
|
@ -21,33 +21,55 @@ import java.io.IOException;
|
|||
|
||||
/** Base implementation class for buffered {@link IndexOutput}. */
|
||||
public abstract class BufferedIndexOutput extends IndexOutput {
|
||||
static final int BUFFER_SIZE = 16384;
|
||||
public static final int DEFAULT_BUFFER_SIZE = 16384;
|
||||
|
||||
private final byte[] buffer = new byte[BUFFER_SIZE];
|
||||
private final int bufferSize;
|
||||
private final byte[] buffer;
|
||||
private long bufferStart = 0; // position in file of buffer
|
||||
private int bufferPosition = 0; // position in buffer
|
||||
|
||||
/**
|
||||
* Creates a new {@link BufferedIndexOutput} with the default buffer size
|
||||
* {@value BufferedIndexOutput#DEFAULT_BUFFER_SIZE}
|
||||
*/
|
||||
public BufferedIndexOutput() {
|
||||
this(DEFAULT_BUFFER_SIZE);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new {@link BufferedIndexOutput} with the given buffer size.
|
||||
* @param bufferSize the buffer size in bytes used to buffer writes internally.
|
||||
* @throws IllegalArgumentException if the given buffer size is less or equal to <tt>0</tt>
|
||||
*/
|
||||
public BufferedIndexOutput(int bufferSize) {
|
||||
if (bufferSize <= 0) {
|
||||
throw new IllegalArgumentException("bufferSize must be greater than 0 (got " + bufferSize + ")");
|
||||
}
|
||||
this.bufferSize = bufferSize;
|
||||
buffer = new byte[bufferSize];
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeByte(byte b) throws IOException {
|
||||
if (bufferPosition >= BUFFER_SIZE)
|
||||
if (bufferPosition >= bufferSize)
|
||||
flush();
|
||||
buffer[bufferPosition++] = b;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeBytes(byte[] b, int offset, int length) throws IOException {
|
||||
int bytesLeft = BUFFER_SIZE - bufferPosition;
|
||||
int bytesLeft = bufferSize - bufferPosition;
|
||||
// is there enough space in the buffer?
|
||||
if (bytesLeft >= length) {
|
||||
// we add the data to the end of the buffer
|
||||
System.arraycopy(b, offset, buffer, bufferPosition, length);
|
||||
bufferPosition += length;
|
||||
// if the buffer is full, flush it
|
||||
if (BUFFER_SIZE - bufferPosition == 0)
|
||||
if (bufferSize - bufferPosition == 0)
|
||||
flush();
|
||||
} else {
|
||||
// is data larger then buffer?
|
||||
if (length > BUFFER_SIZE) {
|
||||
if (length > bufferSize) {
|
||||
// we flush the buffer
|
||||
if (bufferPosition > 0)
|
||||
flush();
|
||||
|
@ -64,10 +86,10 @@ public abstract class BufferedIndexOutput extends IndexOutput {
|
|||
pos += pieceLength;
|
||||
bufferPosition += pieceLength;
|
||||
// if the buffer is full, flush it
|
||||
bytesLeft = BUFFER_SIZE - bufferPosition;
|
||||
bytesLeft = bufferSize - bufferPosition;
|
||||
if (bytesLeft == 0) {
|
||||
flush();
|
||||
bytesLeft = BUFFER_SIZE;
|
||||
bytesLeft = bufferSize;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -439,7 +439,12 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
|
|||
}
|
||||
}
|
||||
//System.out.println(Thread.currentThread().getName() + ": MDW: create " + name);
|
||||
IndexOutput io = new MockIndexOutputWrapper(this, delegate.createOutput(name, LuceneTestCase.newIOContext(randomState, context)), name);
|
||||
IndexOutput delegateOutput = delegate.createOutput(name, LuceneTestCase.newIOContext(randomState, context));
|
||||
if (randomState.nextInt(10) == 0){
|
||||
// once in a while wrap the IO in a Buffered IO with random buffer sizes
|
||||
delegateOutput = new BufferedIndexOutputWrapper(1+randomState.nextInt(BufferedIndexOutput.DEFAULT_BUFFER_SIZE), delegateOutput);
|
||||
}
|
||||
final IndexOutput io = new MockIndexOutputWrapper(this, delegateOutput, name);
|
||||
addFileHandle(io, name, Handle.Output);
|
||||
openFilesForWrite.add(name);
|
||||
|
||||
|
@ -890,4 +895,41 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
|
|||
addFileHandle(handle, name, Handle.Slice);
|
||||
return handle;
|
||||
}
|
||||
|
||||
final class BufferedIndexOutputWrapper extends BufferedIndexOutput {
|
||||
private final IndexOutput io;
|
||||
|
||||
public BufferedIndexOutputWrapper(int bufferSize, IndexOutput io) {
|
||||
super(bufferSize);
|
||||
this.io = io;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long length() throws IOException {
|
||||
return io.length();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void flushBuffer(byte[] b, int offset, int len) throws IOException {
|
||||
io.writeBytes(b, offset, len);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void flush() throws IOException {
|
||||
try {
|
||||
super.flush();
|
||||
} finally {
|
||||
io.flush();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
try {
|
||||
super.close();
|
||||
} finally {
|
||||
io.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue