Index: src/java/org/apache/lucene/index/CompoundFileReader.java =================================================================== --- src/java/org/apache/lucene/index/CompoundFileReader.java (revision 224432) +++ src/java/org/apache/lucene/index/CompoundFileReader.java (working copy) @@ -211,13 +211,10 @@ protected void readInternal(byte[] b, int offset, int len) throws IOException { - synchronized (base) { - long start = getFilePointer(); - if(start + len > length) + final long start = getFilePointer(); + if (start + len > length) throw new IOException("read past EOF"); - base.seek(fileOffset + start); - base.readBytes(b, offset, len); - } + base.readBytes(b, offset, len, fileOffset + start); } /** Expert: implements seek. Sets current position in this file, where Index: src/java/org/apache/lucene/index/FieldsReader.java =================================================================== --- src/java/org/apache/lucene/index/FieldsReader.java (revision 224432) +++ src/java/org/apache/lucene/index/FieldsReader.java (working copy) @@ -37,6 +37,8 @@ private FieldInfos fieldInfos; private IndexInput fieldsStream; private IndexInput indexStream; + private final ThreadStream fstream; + private final ThreadStream istream; private int size; FieldsReader(Directory d, String segment, FieldInfos fn) throws IOException { @@ -45,6 +47,9 @@ fieldsStream = d.openInput(segment + ".fdt"); indexStream = d.openInput(segment + ".fdx"); + fstream = new ThreadStream(fieldsStream); + istream = new ThreadStream(indexStream); + size = (int)(indexStream.length() / 8); } @@ -58,6 +63,9 @@ } final Document doc(int n) throws IOException { + IndexInput indexStream = istream.getStream(); + IndexInput fieldsStream = fstream.getStream(); + indexStream.seek(n * 8L); long position = indexStream.readLong(); fieldsStream.seek(position); @@ -142,4 +150,20 @@ // Get the decompressed data return bos.toByteArray(); } + + /** + * a InputStream safe for use by multiple threads + */ + private static class ThreadStream extends ThreadLocal { + private final IndexInput is; + public ThreadStream(IndexInput is) { + this.is = is; + } + public final IndexInput getStream() { + return (IndexInput) get(); + } + public final Object initialValue() { + return is.clone(); + } + } } Index: src/java/org/apache/lucene/index/SegmentReader.java =================================================================== --- src/java/org/apache/lucene/index/SegmentReader.java (revision 224432) +++ src/java/org/apache/lucene/index/SegmentReader.java (working copy) @@ -277,15 +277,17 @@ return tis.terms(t); } - public synchronized Document document(int n) throws IOException { + public Document document(int n) throws IOException { if (isDeleted(n)) throw new IllegalArgumentException ("attempt to access a deleted document"); return fieldsReader.doc(n); } - public synchronized boolean isDeleted(int n) { - return (deletedDocs != null && deletedDocs.get(n)); + public boolean isDeleted(int n) { + // avoid race condition by getting a snapshot reference + final BitVector snapshot = deletedDocs; + return (snapshot != null && snapshot.get(n)); } public TermDocs termDocs() throws IOException { Index: src/java/org/apache/lucene/store/NioFSDirectory.java =================================================================== --- src/java/org/apache/lucene/store/NioFSDirectory.java (revision 0) +++ src/java/org/apache/lucene/store/NioFSDirectory.java (revision 0) @@ -0,0 +1,80 @@ +package org.apache.lucene.store; + +import java.io.File; +import java.io.IOException; + +/** + * Java NIO implementation of {@link Directory}. + * + *

To use NioFSDirectory, invoke Java with the System property + * org.apache.lucene.FSDirectory.class set to + * org.apache.lucene.store.NioFSDirectory. + * + * This will cause {@link FSDirectory#getDirectory(File, boolean)} + * to return instances of this class. + * + * @see org.apache.lucene.store.MMapDirectory + */ +public class NioFSDirectory extends FSDirectory { + + // default constructor to allow newInstance() calls by FSDirectory + protected NioFSDirectory() {} + + public NioFSDirectory(File path, boolean create) throws IOException { + super(); + init(path, create); + } + + public IndexInput openInput(String name) throws IOException { + return new NioIndexInput(new File(getFile(), name)); + } +} + +final class NioIndexInput extends BufferedIndexInput { + private final org.apache.lucene.store.NioFile file; + private final long length; + private boolean isClone; + + public NioIndexInput(File path) throws IOException { + file = new NioFile(path, "r"); + length = path.length(); + } + + /** + * IndexInput methods + */ + protected final void readInternal(byte[] b, int offset, int len) throws IOException { + readBytes(b, offset, len, getFilePointer()); + } + + public void readBytes(byte[] b, int offset, int len, long position) throws IOException { + file.read(b, offset, len, position); + } + + public final void close() throws IOException { + if (!isClone) + file.close(); + } + + public final long length() { + return length; + } + + protected final void seekInternal(long position) { + } + + protected final void finalize() throws Throwable { + close(); + } + + public Object clone() { + NioIndexInput clone = (NioIndexInput) super.clone(); + clone.isClone = true; + return clone; + } + + boolean isFDValid() { + return file.isOpen(); + } +} + Index: src/java/org/apache/lucene/store/NioFile.java =================================================================== --- src/java/org/apache/lucene/store/NioFile.java (revision 0) +++ src/java/org/apache/lucene/store/NioFile.java (revision 0) @@ -0,0 +1,109 @@ +package org.apache.lucene.store; + +import org.apache.lucene.util.Constants; + +import java.io.*; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.util.*; + +/** + * Wrapper for java.nio.channels.FileChannel that limits the concurrent channel + * operations to specified limit. The limit is set in the system property + * org.apache.lucene.nio.channels, and defaults to 4. + * However, under Windows NioFile only allows a single channel, + * due to JDK bug #6265734. + */ +public class NioFile { + /** + * Number of concurrent channels allowed; specified + * by system property org.apache.lucene.nio.channels. + * Default for non-Windows platforms is 4 concurrent channels. + */ + private static int NCHANNELS; + static { + try { + NCHANNELS = Integer.parseInt(System.getProperty("org.apache.lucene.nio.channels", "4")); + } catch (NumberFormatException nfe) { + NCHANNELS = 4; + } + } + static final boolean SINGLECHANNEL = !Constants.WINDOWS; + + private final File path; + // using a stack instead of a queue may be better for systems + // with little concurrent use + private final LinkedList queue = new LinkedList(); + private int count = 0; + + private final String mode; + private boolean open = true; + + private final FileChannel channel; + + public NioFile(File path, String mode) throws IOException { + this.path = path; + this.mode = mode; + this.channel = new RandomAccessFile(path, mode).getChannel(); + if (!SINGLECHANNEL) + release(channel); + } + + private FileChannel getChannel() throws IOException { + if (SINGLECHANNEL) + return channel; + + synchronized (queue) { + while (queue.size() == 0) { + if (count < NCHANNELS) { + count++; + return new RandomAccessFile(path, mode).getChannel(); + } + try { + queue.wait(); + } catch (InterruptedException e) { + throw new IOException("interrupted"); + } + } + return (FileChannel) queue.removeFirst(); + } + } + + private void release(FileChannel channel) { + if (SINGLECHANNEL) + return; + + synchronized (queue) { + queue.add(channel); + queue.notify(); + } + } + + public void close() throws IOException { + synchronized (queue) { + for (Iterator i = queue.iterator(); i.hasNext();) { + FileChannel fc = (FileChannel) i.next(); + fc.close(); + i.remove(); + } + } + open = false; + } + + public boolean isOpen() { + return open; + } + + public void read(byte[] b, int offset, int len, long position) throws IOException { + read(ByteBuffer.wrap(b, offset, len), position); + } + + private void read(ByteBuffer buffer, long position) throws IOException { + FileChannel channel = getChannel(); + try { + channel.read(buffer, position); + } finally { + release(channel); + } + } +} Index: src/java/org/apache/lucene/store/IndexInput.java =================================================================== --- src/java/org/apache/lucene/store/IndexInput.java (revision 224432) +++ src/java/org/apache/lucene/store/IndexInput.java (working copy) @@ -24,6 +24,10 @@ */ public abstract class IndexInput implements Cloneable { private char[] chars; // used by readString() + /** + * Internal lock held in {@link #readBytes(byte[], int, int, long)}. + */ + private final Object lock = new Object(); /** Reads and returns a single byte. * @see IndexOutput#writeByte(byte) @@ -39,6 +43,25 @@ public abstract void readBytes(byte[] b, int offset, int len) throws IOException; + /** + * Reads a specified number of bytes into an array at the specified offset from a specified position + * in the stream. The current stream position will be moved remain unchanged. Subclasses can override + * this method for increased performance. + * + * @param b the array to read bytes into + * @param offset the offset in the array to start storing bytes + * @param len the number of bytes to read + * @param position the stream offset + */ + public void readBytes(byte[] b, int offset, int len, long position) throws IOException { + synchronized(lock) { + long old = getFilePointer(); + seek(position); + readBytes(b, offset, len); + seek(old); + } + } + /** Reads four bytes and returns an int. * @see IndexOutput#writeInt(int) */ Index: src/java/org/apache/lucene/store/FSDirectory.java =================================================================== --- src/java/org/apache/lucene/store/FSDirectory.java (revision 224432) +++ src/java/org/apache/lucene/store/FSDirectory.java (working copy) @@ -135,9 +135,9 @@ private int refCount; private File lockDir; - protected FSDirectory() {}; // permit subclassing + protected FSDirectory() {} // permit subclassing - private void init(File path, boolean create) throws IOException { + protected void init(File path, boolean create) throws IOException { directory = path; if (LOCK_DIR == null) {