mirror of https://github.com/apache/lucene.git
LUCENE-5164: Fix default chunk sizes in FSDirectory to not be unnecessarily large (now 8192 bytes); also use chunking when writing to index files. FSDirectory#setReadChunkSize() is now deprecated and will be removed in Lucene 5.0
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1512937 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
baea2a94a3
commit
e9e349b32b
|
@ -106,13 +106,10 @@ Bug Fixes
|
|||
seek/lookup which can cause sideeffects if done on a cached FST root arc.
|
||||
(Simon Willnauer)
|
||||
|
||||
* LUCENE-5160: Handle the case where reading from a file or FileChannel returns -1, which
|
||||
could happen in rare cases where something happens to the file between the time we start the
|
||||
read loop (where we check the length) and when we actually do the read. (gsingers, yonik, Robert Muir, Uwe Schindler)
|
||||
|
||||
* LUCENE-5161: Fix default chunk sizes in FSDirectory.java to not be unnecessarily large,
|
||||
and fix setReadChunkSize to always work regardless of whether the machine is 32bit
|
||||
or 64bit. (Uwe Schindler, Robert Muir)
|
||||
* LUCENE-5160: Handle the case where reading from a file or FileChannel returns -1,
|
||||
which could happen in rare cases where something happens to the file between the
|
||||
time we start the read loop (where we check the length) and when we actually do
|
||||
the read. (gsingers, yonik, Robert Muir, Uwe Schindler)
|
||||
|
||||
API Changes
|
||||
|
||||
|
@ -149,6 +146,11 @@ API Changes
|
|||
* LUCENE-5156: CompressingTermVectors TermsEnum no longer supports ord().
|
||||
(Robert Muir)
|
||||
|
||||
* LUCENE-5161, LUCENE-5164: Fix default chunk sizes in FSDirectory to not be
|
||||
unnecessarily large (now 8192 bytes); also use chunking when writing to index
|
||||
files. FSDirectory#setReadChunkSize() is now deprecated and will be removed
|
||||
in Lucene 5.0. (Uwe Schindler, Robert Muir, gsingers)
|
||||
|
||||
Optimizations
|
||||
|
||||
* LUCENE-5088: Added TermFilter to filter docs by a specific term.
|
||||
|
|
|
@ -23,7 +23,7 @@ import java.io.IOException;
|
|||
/** Base implementation class for buffered {@link IndexInput}. */
|
||||
public abstract class BufferedIndexInput extends IndexInput {
|
||||
|
||||
/** Default buffer size set to 1024*/
|
||||
/** Default buffer size set to {@value #BUFFER_SIZE}. */
|
||||
public static final int BUFFER_SIZE = 1024;
|
||||
|
||||
// The normal read buffer size defaults to 1024, but
|
||||
|
@ -33,7 +33,7 @@ public abstract class BufferedIndexInput extends IndexInput {
|
|||
// BufferedIndexInputs created during merging. See
|
||||
// LUCENE-888 for details.
|
||||
/**
|
||||
* A buffer size for merges set to 4096
|
||||
* A buffer size for merges set to {@value #MERGE_BUFFER_SIZE}.
|
||||
*/
|
||||
public static final int MERGE_BUFFER_SIZE = 4096;
|
||||
|
||||
|
@ -115,15 +115,14 @@ public abstract class BufferedIndexInput extends IndexInput {
|
|||
|
||||
@Override
|
||||
public final void readBytes(byte[] b, int offset, int len, boolean useBuffer) throws IOException {
|
||||
|
||||
if(len <= (bufferLength-bufferPosition)){
|
||||
int available = bufferLength - bufferPosition;
|
||||
if(len <= available){
|
||||
// the buffer contains enough data to satisfy this request
|
||||
if(len>0) // to allow b to be null if len is 0...
|
||||
System.arraycopy(buffer, bufferPosition, b, offset, len);
|
||||
bufferPosition+=len;
|
||||
} else {
|
||||
// the buffer does not have enough data. First serve all we've got.
|
||||
int available = bufferLength - bufferPosition;
|
||||
if(available > 0){
|
||||
System.arraycopy(buffer, bufferPosition, b, offset, available);
|
||||
offset += available;
|
||||
|
|
|
@ -31,6 +31,7 @@ import java.util.concurrent.Future;
|
|||
|
||||
import org.apache.lucene.util.ThreadInterruptedException;
|
||||
import org.apache.lucene.util.Constants;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
/**
|
||||
* Base class for Directory implementations that store index
|
||||
|
@ -112,13 +113,16 @@ import org.apache.lucene.util.Constants;
|
|||
public abstract class FSDirectory extends Directory {
|
||||
|
||||
/**
|
||||
* Default read chunk size: 2*{@link BufferedIndexInput#MERGE_BUFFER_SIZE}.
|
||||
* Default read chunk size: 8192 bytes (this is the size up to which the JDK
|
||||
does not allocate additional arrays while reading/writing)
|
||||
@deprecated This constant is no longer used since Lucene 4.5.
|
||||
*/
|
||||
public static final int DEFAULT_READ_CHUNK_SIZE = BufferedIndexInput.MERGE_BUFFER_SIZE * 2;
|
||||
@Deprecated
|
||||
public static final int DEFAULT_READ_CHUNK_SIZE = 8192;
|
||||
|
||||
protected final File directory; // The underlying filesystem directory
|
||||
protected final Set<String> staleFiles = synchronizedSet(new HashSet<String>()); // Files written, but not yet sync'ed
|
||||
private int chunkSize = DEFAULT_READ_CHUNK_SIZE; // LUCENE-1566
|
||||
private int chunkSize = DEFAULT_READ_CHUNK_SIZE;
|
||||
|
||||
// returns the canonical version of the directory, creating it if it doesn't exist.
|
||||
private static File getCanonicalPath(File file) throws IOException {
|
||||
|
@ -352,24 +356,11 @@ public abstract class FSDirectory extends Directory {
|
|||
}
|
||||
|
||||
/**
|
||||
* Sets the maximum number of bytes read at once from the
|
||||
* underlying file during {@link IndexInput#readBytes}.
|
||||
* The default value is {@link #DEFAULT_READ_CHUNK_SIZE};
|
||||
*
|
||||
* <p> This was introduced due to <a
|
||||
* href="http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=6478546">Sun
|
||||
* JVM Bug 6478546</a>, which throws an incorrect
|
||||
* OutOfMemoryError when attempting to read too many bytes
|
||||
* at once. It only happens on 32bit JVMs with a large
|
||||
* maximum heap size.</p>
|
||||
*
|
||||
* <p>Changes to this value will not impact any
|
||||
* already-opened {@link IndexInput}s. You should call
|
||||
* this before attempting to open an index on the
|
||||
* directory.</p>
|
||||
* This setting has no effect anymore.
|
||||
* @deprecated This is no longer used since Lucene 4.5.
|
||||
*/
|
||||
@Deprecated
|
||||
public final void setReadChunkSize(int chunkSize) {
|
||||
// LUCENE-1566
|
||||
if (chunkSize <= 0) {
|
||||
throw new IllegalArgumentException("chunkSize must be positive");
|
||||
}
|
||||
|
@ -377,12 +368,11 @@ public abstract class FSDirectory extends Directory {
|
|||
}
|
||||
|
||||
/**
|
||||
* The maximum number of bytes to read at once from the
|
||||
* underlying file during {@link IndexInput#readBytes}.
|
||||
* @see #setReadChunkSize
|
||||
* This setting has no effect anymore.
|
||||
* @deprecated This is no longer used since Lucene 4.5.
|
||||
*/
|
||||
@Deprecated
|
||||
public final int getReadChunkSize() {
|
||||
// LUCENE-1566
|
||||
return chunkSize;
|
||||
}
|
||||
|
||||
|
@ -390,23 +380,35 @@ public abstract class FSDirectory extends Directory {
|
|||
* Writes output with {@link RandomAccessFile#write(byte[], int, int)}
|
||||
*/
|
||||
protected static class FSIndexOutput extends BufferedIndexOutput {
|
||||
/**
|
||||
* The maximum chunk size is 8192 bytes, because {@link RandomAccessFile} mallocs
|
||||
* a native buffer outside of stack if the write buffer size is larger.
|
||||
*/
|
||||
private static final int CHUNK_SIZE = 8192;
|
||||
|
||||
private final FSDirectory parent;
|
||||
private final String name;
|
||||
private final RandomAccessFile file;
|
||||
private volatile boolean isOpen; // remember if the file is open, so that we don't try to close it more than once
|
||||
|
||||
public FSIndexOutput(FSDirectory parent, String name) throws IOException {
|
||||
super(CHUNK_SIZE);
|
||||
this.parent = parent;
|
||||
this.name = name;
|
||||
file = new RandomAccessFile(new File(parent.directory, name), "rw");
|
||||
isOpen = true;
|
||||
}
|
||||
|
||||
/** output methods: */
|
||||
@Override
|
||||
public void flushBuffer(byte[] b, int offset, int size) throws IOException {
|
||||
protected void flushBuffer(byte[] b, int offset, int size) throws IOException {
|
||||
assert isOpen;
|
||||
file.write(b, offset, size);
|
||||
while (size > 0) {
|
||||
final int toWrite = Math.min(CHUNK_SIZE, size);
|
||||
file.write(b, offset, toWrite);
|
||||
offset += toWrite;
|
||||
size -= toWrite;
|
||||
}
|
||||
assert size == 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -414,21 +416,14 @@ public abstract class FSDirectory extends Directory {
|
|||
parent.onIndexOutputClosed(name);
|
||||
// only close the file if it has not been closed yet
|
||||
if (isOpen) {
|
||||
boolean success = false;
|
||||
IOException priorE = null;
|
||||
try {
|
||||
super.close();
|
||||
success = true;
|
||||
} catch (IOException ioe) {
|
||||
priorE = ioe;
|
||||
} finally {
|
||||
isOpen = false;
|
||||
if (!success) {
|
||||
try {
|
||||
file.close();
|
||||
} catch (Throwable t) {
|
||||
// Suppress so we don't mask original exception
|
||||
}
|
||||
} else {
|
||||
file.close();
|
||||
}
|
||||
IOUtils.closeWhileHandlingException(priorE, file);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,6 +20,7 @@ package org.apache.lucene.store;
|
|||
import java.io.File;
|
||||
import java.io.EOFException;
|
||||
import java.io.IOException;
|
||||
import java.io.RandomAccessFile;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.channels.ClosedChannelException; // javadoc @link
|
||||
import java.nio.channels.FileChannel;
|
||||
|
@ -79,7 +80,7 @@ public class NIOFSDirectory extends FSDirectory {
|
|||
ensureOpen();
|
||||
File path = new File(getDirectory(), name);
|
||||
FileChannel fc = FileChannel.open(path.toPath(), StandardOpenOption.READ);
|
||||
return new NIOFSIndexInput("NIOFSIndexInput(path=\"" + path + "\")", fc, context, getReadChunkSize());
|
||||
return new NIOFSIndexInput("NIOFSIndexInput(path=\"" + path + "\")", fc, context);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -98,7 +99,7 @@ public class NIOFSDirectory extends FSDirectory {
|
|||
@Override
|
||||
public IndexInput openSlice(String sliceDescription, long offset, long length) {
|
||||
return new NIOFSIndexInput("NIOFSIndexInput(" + sliceDescription + " in path=\"" + path + "\" slice=" + offset + ":" + (offset+length) + ")", descriptor, offset,
|
||||
length, BufferedIndexInput.bufferSize(context), getReadChunkSize());
|
||||
length, BufferedIndexInput.bufferSize(context));
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -107,12 +108,15 @@ public class NIOFSDirectory extends FSDirectory {
|
|||
* Reads bytes with {@link FileChannel#read(ByteBuffer, long)}
|
||||
*/
|
||||
protected static class NIOFSIndexInput extends BufferedIndexInput {
|
||||
/**
|
||||
* The maximum chunk size for reads of 16384 bytes.
|
||||
*/
|
||||
private static final int CHUNK_SIZE = 16384;
|
||||
|
||||
/** the file channel we will read from */
|
||||
protected final FileChannel channel;
|
||||
/** is this instance a clone and hence does not own the file to close it */
|
||||
boolean isClone = false;
|
||||
/** maximum read length on a 32bit JVM to prevent incorrect OOM, see LUCENE-1566 */
|
||||
protected final int chunkSize;
|
||||
/** start offset: non-zero in the slice case */
|
||||
protected final long off;
|
||||
/** end offset (start+length) */
|
||||
|
@ -120,18 +124,16 @@ public class NIOFSDirectory extends FSDirectory {
|
|||
|
||||
private ByteBuffer byteBuf; // wraps the buffer for NIO
|
||||
|
||||
public NIOFSIndexInput(String resourceDesc, FileChannel fc, IOContext context, int chunkSize) throws IOException {
|
||||
public NIOFSIndexInput(String resourceDesc, FileChannel fc, IOContext context) throws IOException {
|
||||
super(resourceDesc, context);
|
||||
this.channel = fc;
|
||||
this.chunkSize = chunkSize;
|
||||
this.off = 0L;
|
||||
this.end = fc.size();
|
||||
}
|
||||
|
||||
public NIOFSIndexInput(String resourceDesc, FileChannel fc, long off, long length, int bufferSize, int chunkSize) {
|
||||
public NIOFSIndexInput(String resourceDesc, FileChannel fc, long off, long length, int bufferSize) {
|
||||
super(resourceDesc, bufferSize);
|
||||
this.channel = fc;
|
||||
this.chunkSize = chunkSize;
|
||||
this.off = off;
|
||||
this.end = off + length;
|
||||
this.isClone = true;
|
||||
|
@ -164,24 +166,18 @@ public class NIOFSDirectory extends FSDirectory {
|
|||
|
||||
@Override
|
||||
protected void readInternal(byte[] b, int offset, int len) throws IOException {
|
||||
|
||||
final ByteBuffer bb;
|
||||
|
||||
// Determine the ByteBuffer we should use
|
||||
if (b == buffer && 0 == offset) {
|
||||
if (b == buffer) {
|
||||
// Use our own pre-wrapped byteBuf:
|
||||
assert byteBuf != null;
|
||||
byteBuf.clear();
|
||||
byteBuf.limit(len);
|
||||
bb = byteBuf;
|
||||
byteBuf.clear().position(offset);
|
||||
} else {
|
||||
bb = ByteBuffer.wrap(b, offset, len);
|
||||
}
|
||||
|
||||
int readOffset = bb.position();
|
||||
int readLength = bb.limit() - readOffset;
|
||||
assert readLength == len;
|
||||
|
||||
long pos = getFilePointer() + off;
|
||||
|
||||
if (pos + len > end) {
|
||||
|
@ -189,33 +185,20 @@ public class NIOFSDirectory extends FSDirectory {
|
|||
}
|
||||
|
||||
try {
|
||||
int readLength = len;
|
||||
while (readLength > 0) {
|
||||
final int limit;
|
||||
if (readLength > chunkSize) {
|
||||
// LUCENE-1566 - work around JVM Bug by breaking
|
||||
// very large reads into chunks
|
||||
limit = readOffset + chunkSize;
|
||||
} else {
|
||||
limit = readOffset + readLength;
|
||||
}
|
||||
bb.limit(limit);
|
||||
int i = channel.read(bb, pos);
|
||||
if (i < 0){//be defensive here, even though we checked before hand, something could have changed
|
||||
throw new EOFException("read past EOF: " + this + " off: " + offset + " len: " + len + " pos: " + pos + " limit: " + limit + " end: " + end);
|
||||
final int toRead = Math.min(CHUNK_SIZE, readLength);
|
||||
bb.limit(bb.position() + toRead);
|
||||
assert bb.remaining() == toRead;
|
||||
final int i = channel.read(bb, pos);
|
||||
if (i < 0) { // be defensive here, even though we checked before hand, something could have changed
|
||||
throw new EOFException("read past EOF: " + this + " off: " + offset + " len: " + len + " pos: " + pos + " chunkLen: " + toRead + " end: " + end);
|
||||
}
|
||||
assert i > 0 : "FileChannel.read with non zero-length bb.remaining() must always read at least one byte (FileChannel is in blocking mode, see spec of ReadableByteChannel)";
|
||||
pos += i;
|
||||
readOffset += i;
|
||||
readLength -= i;
|
||||
}
|
||||
} catch (OutOfMemoryError e) {
|
||||
// propagate OOM up and add a hint for 32bit VM Users hitting the bug
|
||||
// with a large chunk size in the fast path.
|
||||
final OutOfMemoryError outOfMemoryError = new OutOfMemoryError(
|
||||
"OutOfMemoryError likely caused by the Sun VM Bug described in "
|
||||
+ "https://issues.apache.org/jira/browse/LUCENE-1566; try calling FSDirectory.setReadChunkSize "
|
||||
+ "with a value smaller than the current chunk size (" + chunkSize + ")");
|
||||
outOfMemoryError.initCause(e);
|
||||
throw outOfMemoryError;
|
||||
assert readLength == 0;
|
||||
} catch (IOException ioe) {
|
||||
throw new IOException(ioe.getMessage() + ": " + this, ioe);
|
||||
}
|
||||
|
|
|
@ -56,7 +56,7 @@ public class SimpleFSDirectory extends FSDirectory {
|
|||
ensureOpen();
|
||||
final File path = new File(directory, name);
|
||||
RandomAccessFile raf = new RandomAccessFile(path, "r");
|
||||
return new SimpleFSIndexInput("SimpleFSIndexInput(path=\"" + path.getPath() + "\")", raf, context, getReadChunkSize());
|
||||
return new SimpleFSIndexInput("SimpleFSIndexInput(path=\"" + path.getPath() + "\")", raf, context);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -75,7 +75,7 @@ public class SimpleFSDirectory extends FSDirectory {
|
|||
@Override
|
||||
public IndexInput openSlice(String sliceDescription, long offset, long length) {
|
||||
return new SimpleFSIndexInput("SimpleFSIndexInput(" + sliceDescription + " in path=\"" + file.getPath() + "\" slice=" + offset + ":" + (offset+length) + ")", descriptor, offset,
|
||||
length, BufferedIndexInput.bufferSize(context), getReadChunkSize());
|
||||
length, BufferedIndexInput.bufferSize(context));
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -85,29 +85,31 @@ public class SimpleFSDirectory extends FSDirectory {
|
|||
* {@link RandomAccessFile#read(byte[], int, int)}.
|
||||
*/
|
||||
protected static class SimpleFSIndexInput extends BufferedIndexInput {
|
||||
/**
|
||||
* The maximum chunk size is 8192 bytes, because {@link RandomAccessFile} mallocs
|
||||
* a native buffer outside of stack if the read buffer size is larger.
|
||||
*/
|
||||
private static final int CHUNK_SIZE = 8192;
|
||||
|
||||
/** the file channel we will read from */
|
||||
protected final RandomAccessFile file;
|
||||
/** is this instance a clone and hence does not own the file to close it */
|
||||
boolean isClone = false;
|
||||
/** maximum read length on a 32bit JVM to prevent incorrect OOM, see LUCENE-1566 */
|
||||
protected final int chunkSize;
|
||||
/** start offset: non-zero in the slice case */
|
||||
protected final long off;
|
||||
/** end offset (start+length) */
|
||||
protected final long end;
|
||||
|
||||
public SimpleFSIndexInput(String resourceDesc, RandomAccessFile file, IOContext context, int chunkSize) throws IOException {
|
||||
public SimpleFSIndexInput(String resourceDesc, RandomAccessFile file, IOContext context) throws IOException {
|
||||
super(resourceDesc, context);
|
||||
this.file = file;
|
||||
this.chunkSize = chunkSize;
|
||||
this.off = 0L;
|
||||
this.end = file.length();
|
||||
}
|
||||
|
||||
public SimpleFSIndexInput(String resourceDesc, RandomAccessFile file, long off, long length, int bufferSize, int chunkSize) {
|
||||
public SimpleFSIndexInput(String resourceDesc, RandomAccessFile file, long off, long length, int bufferSize) {
|
||||
super(resourceDesc, bufferSize);
|
||||
this.file = file;
|
||||
this.chunkSize = chunkSize;
|
||||
this.off = off;
|
||||
this.end = off + length;
|
||||
this.isClone = true;
|
||||
|
@ -146,29 +148,16 @@ public class SimpleFSDirectory extends FSDirectory {
|
|||
}
|
||||
|
||||
try {
|
||||
do {
|
||||
final int readLength;
|
||||
if (total + chunkSize > len) {
|
||||
readLength = len - total;
|
||||
} else {
|
||||
// LUCENE-1566 - work around JVM Bug by breaking very large reads into chunks
|
||||
readLength = chunkSize;
|
||||
}
|
||||
final int i = file.read(b, offset + total, readLength);
|
||||
if (i < 0){//be defensive here, even though we checked before hand, something could have changed
|
||||
throw new EOFException("read past EOF: " + this + " off: " + offset + " len: " + len + " total: " + total + " readLen: " + readLength + " end: " + end);
|
||||
while (total < len) {
|
||||
final int toRead = Math.min(CHUNK_SIZE, len - total);
|
||||
final int i = file.read(b, offset + total, toRead);
|
||||
if (i < 0) { // be defensive here, even though we checked before hand, something could have changed
|
||||
throw new EOFException("read past EOF: " + this + " off: " + offset + " len: " + len + " total: " + total + " chunkLen: " + toRead + " end: " + end);
|
||||
}
|
||||
assert i > 0 : "RandomAccessFile.read with non zero-length toRead must always read at least one byte";
|
||||
total += i;
|
||||
} while (total < len);
|
||||
} catch (OutOfMemoryError e) {
|
||||
// propagate OOM up and add a hint for 32bit VM Users hitting the bug
|
||||
// with a large chunk size in the fast path.
|
||||
final OutOfMemoryError outOfMemoryError = new OutOfMemoryError(
|
||||
"OutOfMemoryError likely caused by the Sun VM Bug described in "
|
||||
+ "https://issues.apache.org/jira/browse/LUCENE-1566; try calling FSDirectory.setReadChunkSize "
|
||||
+ "with a value smaller than the current chunk size (" + chunkSize + ")");
|
||||
outOfMemoryError.initCause(e);
|
||||
throw outOfMemoryError;
|
||||
}
|
||||
assert total == len;
|
||||
} catch (IOException ioe) {
|
||||
throw new IOException(ioe.getMessage() + ": " + this, ioe);
|
||||
}
|
||||
|
|
|
@ -84,25 +84,6 @@ public class TestBufferedIndexInput extends LuceneTestCase {
|
|||
public void testReadBytes() throws Exception {
|
||||
MyBufferedIndexInput input = new MyBufferedIndexInput();
|
||||
runReadBytes(input, BufferedIndexInput.BUFFER_SIZE, random());
|
||||
|
||||
// This tests the workaround code for LUCENE-1566 where readBytesInternal
|
||||
// provides a workaround for a JVM Bug that incorrectly raises a OOM Error
|
||||
// when a large byte buffer is passed to a file read.
|
||||
// NOTE: this does only test the chunked reads and NOT if the Bug is triggered.
|
||||
//final int tmpFileSize = 1024 * 1024 * 5;
|
||||
final int inputBufferSize = 128;
|
||||
File tmpInputFile = _TestUtil.createTempFile("IndexInput", "tmpFile", TEMP_DIR);
|
||||
tmpInputFile.deleteOnExit();
|
||||
writeBytes(tmpInputFile, TEST_FILE_LENGTH);
|
||||
|
||||
// run test with chunk size of 10 bytes
|
||||
runReadBytesAndClose(new SimpleFSIndexInput("SimpleFSIndexInput(path=\"" + tmpInputFile + "\")",
|
||||
new RandomAccessFile(tmpInputFile, "r"), newIOContext(random()), 10), inputBufferSize, random());
|
||||
|
||||
// run test with chunk size of 10 bytes
|
||||
runReadBytesAndClose(new NIOFSIndexInput("NIOFSIndexInput(path=\"" + tmpInputFile + "\")",
|
||||
FileChannel.open(tmpInputFile.toPath(), StandardOpenOption.READ), newIOContext(random()), 10),
|
||||
inputBufferSize, random());
|
||||
}
|
||||
|
||||
private void runReadBytesAndClose(IndexInput input, int bufferSize, Random r) throws IOException {
|
||||
|
@ -211,6 +192,7 @@ public class TestBufferedIndexInput extends LuceneTestCase {
|
|||
private static byte byten(long n){
|
||||
return (byte)(n*n%256);
|
||||
}
|
||||
|
||||
private static class MyBufferedIndexInput extends BufferedIndexInput {
|
||||
private long pos;
|
||||
private long len;
|
||||
|
|
|
@ -134,52 +134,66 @@ public class TestDirectory extends LuceneTestCase {
|
|||
// Test that different instances of FSDirectory can coexist on the same
|
||||
// path, can read, write, and lock files.
|
||||
public void testDirectInstantiation() throws Exception {
|
||||
File path = _TestUtil.getTempDir("testDirectInstantiation");
|
||||
final File path = _TestUtil.getTempDir("testDirectInstantiation");
|
||||
|
||||
final byte[] largeBuffer = new byte[random().nextInt(256*1024)], largeReadBuffer = new byte[largeBuffer.length];
|
||||
for (int i = 0; i < largeBuffer.length; i++) {
|
||||
largeBuffer[i] = (byte) i; // automatically loops with modulo
|
||||
}
|
||||
|
||||
int sz = 3;
|
||||
Directory[] dirs = new Directory[sz];
|
||||
final FSDirectory[] dirs = new FSDirectory[] {
|
||||
new SimpleFSDirectory(path, null),
|
||||
new NIOFSDirectory(path, null),
|
||||
new MMapDirectory(path, null)
|
||||
};
|
||||
|
||||
dirs[0] = new SimpleFSDirectory(path, null);
|
||||
dirs[1] = new NIOFSDirectory(path, null);
|
||||
dirs[2] = new MMapDirectory(path, null);
|
||||
|
||||
for (int i=0; i<sz; i++) {
|
||||
Directory dir = dirs[i];
|
||||
for (int i=0; i<dirs.length; i++) {
|
||||
FSDirectory dir = dirs[i];
|
||||
dir.ensureOpen();
|
||||
String fname = "foo." + i;
|
||||
String lockname = "foo" + i + ".lck";
|
||||
IndexOutput out = dir.createOutput(fname, newIOContext(random()));
|
||||
out.writeByte((byte)i);
|
||||
out.writeBytes(largeBuffer, largeBuffer.length);
|
||||
out.close();
|
||||
|
||||
for (int j=0; j<sz; j++) {
|
||||
Directory d2 = dirs[j];
|
||||
for (int j=0; j<dirs.length; j++) {
|
||||
FSDirectory d2 = dirs[j];
|
||||
d2.ensureOpen();
|
||||
assertTrue(d2.fileExists(fname));
|
||||
assertEquals(1, d2.fileLength(fname));
|
||||
assertEquals(1 + largeBuffer.length, d2.fileLength(fname));
|
||||
|
||||
// don't test read on MMapDirectory, since it can't really be
|
||||
// closed and will cause a failure to delete the file.
|
||||
if (d2 instanceof MMapDirectory) continue;
|
||||
// don't do read tests if unmapping is not supported!
|
||||
if (d2 instanceof MMapDirectory && !((MMapDirectory) d2).getUseUnmap())
|
||||
continue;
|
||||
|
||||
IndexInput input = d2.openInput(fname, newIOContext(random()));
|
||||
assertEquals((byte)i, input.readByte());
|
||||
// read array with buffering enabled
|
||||
Arrays.fill(largeReadBuffer, (byte)0);
|
||||
input.readBytes(largeReadBuffer, 0, largeReadBuffer.length, true);
|
||||
assertArrayEquals(largeBuffer, largeReadBuffer);
|
||||
// read again without using buffer
|
||||
input.seek(1L);
|
||||
Arrays.fill(largeReadBuffer, (byte)0);
|
||||
input.readBytes(largeReadBuffer, 0, largeReadBuffer.length, false);
|
||||
assertArrayEquals(largeBuffer, largeReadBuffer);
|
||||
input.close();
|
||||
}
|
||||
|
||||
// delete with a different dir
|
||||
dirs[(i+1)%sz].deleteFile(fname);
|
||||
dirs[(i+1)%dirs.length].deleteFile(fname);
|
||||
|
||||
for (int j=0; j<sz; j++) {
|
||||
Directory d2 = dirs[j];
|
||||
for (int j=0; j<dirs.length; j++) {
|
||||
FSDirectory d2 = dirs[j];
|
||||
assertFalse(d2.fileExists(fname));
|
||||
}
|
||||
|
||||
Lock lock = dir.makeLock(lockname);
|
||||
assertTrue(lock.obtain());
|
||||
|
||||
for (int j=0; j<sz; j++) {
|
||||
Directory d2 = dirs[j];
|
||||
for (int j=0; j<dirs.length; j++) {
|
||||
FSDirectory d2 = dirs[j];
|
||||
Lock lock2 = d2.makeLock(lockname);
|
||||
try {
|
||||
assertFalse(lock2.obtain(1));
|
||||
|
@ -191,13 +205,13 @@ public class TestDirectory extends LuceneTestCase {
|
|||
lock.release();
|
||||
|
||||
// now lock with different dir
|
||||
lock = dirs[(i+1)%sz].makeLock(lockname);
|
||||
lock = dirs[(i+1)%dirs.length].makeLock(lockname);
|
||||
assertTrue(lock.obtain());
|
||||
lock.release();
|
||||
}
|
||||
|
||||
for (int i=0; i<sz; i++) {
|
||||
Directory dir = dirs[i];
|
||||
for (int i=0; i<dirs.length; i++) {
|
||||
FSDirectory dir = dirs[i];
|
||||
dir.ensureOpen();
|
||||
dir.close();
|
||||
assertFalse(dir.isOpen);
|
||||
|
|
|
@ -1139,7 +1139,6 @@ public abstract class LuceneTestCase extends Assert {
|
|||
} catch (NoSuchMethodException | InstantiationException | IllegalAccessException | InvocationTargetException e) {
|
||||
Rethrow.rethrow(e);
|
||||
}
|
||||
d.setReadChunkSize(_TestUtil.nextInt(random(), 8, 32678));
|
||||
return d;
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue