LUCENE-2793: Directory#createOutput & Directory#openInput now accept an IOContext instead of a buffer size to allow low level optimizations for different usecases like merging, flushing and reading.

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1144196 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Simon Willnauer 2011-07-08 08:52:40 +00:00
commit c49d2d5d84
143 changed files with 1376 additions and 826 deletions

View File

@ -462,6 +462,15 @@ New features
IndexSearcher. SortFields can have SortField.REWRITEABLE type which
requires they are rewritten before they are used. (Chris Male)
* LUCENE-3203: FSDirectory can now limit the max allowed write rate
(MB/sec) of all running merges, to reduce impact ongoing merging has
on searching, NRT reopen time, etc. (Mike McCandless)
* LUCENE-2793: Directory#createOutput & Directory#openInput now accept an
IOContext instead of a buffer size to allow low level optimizations for
different usecases like merging, flushing and reading.
(Simon Willnauer, Mike McCandless, Varun Thacker)
Optimizations
* LUCENE-2588: Don't store unnecessary suffixes when writing the terms

View File

@ -93,7 +93,7 @@ public class AppendingCodec extends Codec {
@Override
public FieldsProducer fieldsProducer(SegmentReadState state)
throws IOException {
PostingsReaderBase docsReader = new StandardPostingsReader(state.dir, state.segmentInfo, state.readBufferSize, state.codecId);
PostingsReaderBase docsReader = new StandardPostingsReader(state.dir, state.segmentInfo, state.context, state.codecId);
TermsIndexReaderBase indexReader;
boolean success = false;
@ -103,7 +103,7 @@ public class AppendingCodec extends Codec {
state.segmentInfo.name,
state.termsIndexDivisor,
BytesRef.getUTF8SortedAsUnicodeComparator(),
state.codecId);
state.codecId, state.context);
success = true;
} finally {
if (!success) {
@ -115,7 +115,7 @@ public class AppendingCodec extends Codec {
FieldsProducer ret = new AppendingTermsDictReader(indexReader,
state.dir, state.fieldInfos, state.segmentInfo.name,
docsReader,
state.readBufferSize,
state.context,
StandardCodec.TERMS_CACHE_SIZE,
state.codecId);
success = true;
@ -153,6 +153,6 @@ public class AppendingCodec extends Codec {
@Override
public PerDocValues docsProducer(SegmentReadState state) throws IOException {
return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId, getDocValuesUseCFS(), getDocValuesSortComparator());
return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId, getDocValuesUseCFS(), getDocValuesSortComparator(), state.context);
}
}

View File

@ -22,6 +22,7 @@ import java.io.IOException;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.codecs.DefaultSegmentInfosReader;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
public class AppendingSegmentInfosReader extends DefaultSegmentInfosReader {
@ -33,9 +34,9 @@ public class AppendingSegmentInfosReader extends DefaultSegmentInfosReader {
}
@Override
public IndexInput openInput(Directory dir, String segmentsFileName)
public IndexInput openInput(Directory dir, String segmentsFileName, IOContext context)
throws IOException {
return dir.openInput(segmentsFileName);
return dir.openInput(segmentsFileName, context);
}
}

View File

@ -21,14 +21,15 @@ import java.io.IOException;
import org.apache.lucene.index.codecs.DefaultSegmentInfosWriter;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
public class AppendingSegmentInfosWriter extends DefaultSegmentInfosWriter {
@Override
protected IndexOutput createOutput(Directory dir, String segmentsFileName)
protected IndexOutput createOutput(Directory dir, String segmentsFileName, IOContext context)
throws IOException {
return dir.createOutput(segmentsFileName);
return dir.createOutput(segmentsFileName, context);
}
@Override

View File

@ -25,6 +25,7 @@ import org.apache.lucene.index.codecs.BlockTermsReader;
import org.apache.lucene.index.codecs.BlockTermsWriter;
import org.apache.lucene.index.codecs.TermsIndexReaderBase;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.CodecUtil;
@ -32,9 +33,9 @@ public class AppendingTermsDictReader extends BlockTermsReader {
public AppendingTermsDictReader(TermsIndexReaderBase indexReader,
Directory dir, FieldInfos fieldInfos, String segment,
PostingsReaderBase postingsReader, int readBufferSize,
PostingsReaderBase postingsReader, IOContext context,
int termsCacheSize, int codecId) throws IOException {
super(indexReader, dir, fieldInfos, segment, postingsReader, readBufferSize,
super(indexReader, dir, fieldInfos, segment, postingsReader, context,
termsCacheSize, codecId);
}

View File

@ -23,6 +23,7 @@ import java.util.Comparator;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.codecs.FixedGapTermsIndexReader;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CodecUtil;
@ -30,9 +31,9 @@ import org.apache.lucene.util.CodecUtil;
public class AppendingTermsIndexReader extends FixedGapTermsIndexReader {
public AppendingTermsIndexReader(Directory dir, FieldInfos fieldInfos,
String segment, int indexDivisor, Comparator<BytesRef> termComp, int codecId)
String segment, int indexDivisor, Comparator<BytesRef> termComp, int codecId, IOContext context)
throws IOException {
super(dir, fieldInfos, segment, indexDivisor, termComp, codecId);
super(dir, fieldInfos, segment, indexDivisor, termComp, codecId, context);
}
@Override

View File

@ -69,16 +69,22 @@ public class DirectIOLinuxDirectory extends FSDirectory {
}
@Override
public IndexInput openInput(String name, int bufferSize) throws IOException {
public IndexInput openInput(String name, IOContext context) throws IOException {
ensureOpen();
return new DirectIOLinuxIndexInput(new File(getDirectory(), name), forcedBufferSize == 0 ? bufferSize : forcedBufferSize);
return new DirectIOLinuxIndexInput(new File(getDirectory(), name),
bufferSize(context));
}
@Override
public IndexOutput createOutput(String name) throws IOException {
public IndexOutput createOutput(String name, IOContext context) throws IOException {
ensureOpen();
ensureCanWrite(name);
return new DirectIOLinuxIndexOutput(new File(getDirectory(), name), forcedBufferSize == 0 ? BufferedIndexOutput.BUFFER_SIZE : forcedBufferSize);
return new DirectIOLinuxIndexOutput(new File(getDirectory(), name), bufferSize(context));
}
private int bufferSize(IOContext context) {
return forcedBufferSize != 0 ? forcedBufferSize : BufferedIndexInput
.bufferSize(context);
}
private final static class DirectIOLinuxIndexOutput extends IndexOutput {
@ -238,6 +244,7 @@ public class DirectIOLinuxDirectory extends FSDirectory {
private int bufferPos;
public DirectIOLinuxIndexInput(File path, int bufferSize) throws IOException {
// TODO make use of IOContext
FileDescriptor fd = NativePosixUtil.open_direct(path.toString(), true);
fis = new FileInputStream(fd);
channel = fis.getChannel();

View File

@ -21,13 +21,8 @@ import java.io.IOException;
import java.util.Collection;
import java.util.HashSet;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.lucene.index.ConcurrentMergeScheduler;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexWriter; // javadocs
import org.apache.lucene.index.MergePolicy;
import org.apache.lucene.index.MergeScheduler;
import org.apache.lucene.store.RAMDirectory; // javadocs
import org.apache.lucene.util.IOUtils;
@ -38,11 +33,7 @@ import org.apache.lucene.util.IOUtils;
/**
* Wraps a {@link RAMDirectory}
* around any provided delegate directory, to
* be used during NRT search. Make sure you pull the merge
* scheduler using {@link #getMergeScheduler} and pass that to your
* {@link IndexWriter}; this class uses that to keep track of which
* merges are being done by which threads, to decide when to
* cache each written file.
* be used during NRT search.
*
* <p>This class is likely only useful in a near-real-time
* context, where indexing rate is lowish but reopen
@ -54,20 +45,12 @@ import org.apache.lucene.util.IOUtils;
* <p>This is safe to use: when your app calls {IndexWriter#commit},
* all cached files will be flushed from the cached and sync'd.</p>
*
* <p><b>NOTE</b>: this class is somewhat sneaky in its
* approach for spying on merges to determine the size of a
* merge: it records which threads are running which merges
* by watching ConcurrentMergeScheduler's doMerge method.
* While this works correctly, likely future versions of
* this class will take a more general approach.
*
* <p>Here's a simple example usage:
*
* <pre>
* Directory fsDir = FSDirectory.open(new File("/path/to/index"));
* NRTCachingDirectory cachedFSDir = new NRTCachingDirectory(fsDir, 5.0, 60.0);
* IndexWriterConfig conf = new IndexWriterConfig(Version.LUCENE_32, analyzer);
* conf.setMergeScheduler(cachedFSDir.getMergeScheduler());
* IndexWriter writer = new IndexWriter(cachedFSDir, conf);
* </pre>
*
@ -193,17 +176,17 @@ public class NRTCachingDirectory extends Directory {
}
@Override
public IndexOutput createOutput(String name) throws IOException {
public IndexOutput createOutput(String name, IOContext context) throws IOException {
if (VERBOSE) {
System.out.println("nrtdir.createOutput name=" + name);
}
if (doCacheWrite(name)) {
if (doCacheWrite(name, context)) {
if (VERBOSE) {
System.out.println(" to cache");
}
return cache.createOutput(name);
return cache.createOutput(name, context);
} else {
return delegate.createOutput(name);
return delegate.createOutput(name, context);
}
}
@ -219,7 +202,7 @@ public class NRTCachingDirectory extends Directory {
}
@Override
public synchronized IndexInput openInput(String name) throws IOException {
public synchronized IndexInput openInput(String name, IOContext context) throws IOException {
if (VERBOSE) {
System.out.println("nrtdir.openInput name=" + name);
}
@ -227,39 +210,31 @@ public class NRTCachingDirectory extends Directory {
if (VERBOSE) {
System.out.println(" from cache");
}
return cache.openInput(name);
return cache.openInput(name, context);
} else {
return delegate.openInput(name);
return delegate.openInput(name, context);
}
}
@Override
public synchronized CompoundFileDirectory openCompoundInput(String name, int bufferSize) throws IOException {
public synchronized CompoundFileDirectory openCompoundInput(String name, IOContext context) throws IOException {
if (cache.fileExists(name)) {
return cache.openCompoundInput(name, bufferSize);
return cache.openCompoundInput(name, context);
} else {
return delegate.openCompoundInput(name, bufferSize);
return delegate.openCompoundInput(name, context);
}
}
@Override
public synchronized CompoundFileDirectory createCompoundOutput(String name)
public synchronized CompoundFileDirectory createCompoundOutput(String name, IOContext context)
throws IOException {
if (cache.fileExists(name)) {
throw new IOException("File " + name + "already exists");
} else {
return delegate.createCompoundOutput(name);
return delegate.createCompoundOutput(name, context);
}
}
@Override
public synchronized IndexInput openInput(String name, int bufferSize) throws IOException {
if (cache.fileExists(name)) {
return cache.openInput(name, bufferSize);
} else {
return delegate.openInput(name, bufferSize);
}
}
/** Close this directory, which flushes any cached files
* to the delegate and then closes the delegate. */
@ -272,36 +247,21 @@ public class NRTCachingDirectory extends Directory {
delegate.close();
}
private final ConcurrentHashMap<Thread,MergePolicy.OneMerge> merges = new ConcurrentHashMap<Thread,MergePolicy.OneMerge>();
public MergeScheduler getMergeScheduler() {
return new ConcurrentMergeScheduler() {
@Override
protected void doMerge(MergePolicy.OneMerge merge) throws IOException {
try {
merges.put(Thread.currentThread(), merge);
super.doMerge(merge);
} finally {
merges.remove(Thread.currentThread());
}
}
};
}
/** Subclass can override this to customize logic; return
* true if this file should be written to the RAMDirectory. */
protected boolean doCacheWrite(String name) {
final MergePolicy.OneMerge merge = merges.get(Thread.currentThread());
protected boolean doCacheWrite(String name, IOContext context) {
final MergeInfo merge = context.mergeInfo;
//System.out.println(Thread.currentThread().getName() + ": CACHE check merge=" + merge + " size=" + (merge==null ? 0 : merge.estimatedMergeBytes));
return !name.equals(IndexFileNames.SEGMENTS_GEN) && (merge == null || merge.estimatedMergeBytes <= maxMergeSizeBytes) && cache.sizeInBytes() <= maxCachedBytes;
}
private void unCache(String fileName) throws IOException {
final IndexOutput out;
IOContext context = IOContext.DEFAULT;
synchronized(this) {
if (!delegate.fileExists(fileName)) {
assert cache.fileExists(fileName);
out = delegate.createOutput(fileName);
out = delegate.createOutput(fileName, context);
} else {
out = null;
}
@ -310,7 +270,7 @@ public class NRTCachingDirectory extends Directory {
if (out != null) {
IndexInput in = null;
try {
in = cache.openInput(fileName);
in = cache.openInput(fileName, context);
in.copyBytes(out, in.length());
} finally {
IOUtils.closeSafely(false, in, out);

View File

@ -19,6 +19,7 @@ package org.apache.lucene.store;
import java.io.File;
import java.io.IOException;
import org.apache.lucene.store.Directory; // javadoc
import org.apache.lucene.store.NativeFSLockFactory; // javadoc
@ -67,9 +68,9 @@ public class WindowsDirectory extends FSDirectory {
}
@Override
public IndexInput openInput(String name, int bufferSize) throws IOException {
public IndexInput openInput(String name, IOContext context) throws IOException {
ensureOpen();
return new WindowsIndexInput(new File(getDirectory(), name), Math.max(bufferSize, DEFAULT_BUFFERSIZE));
return new WindowsIndexInput(new File(getDirectory(), name), Math.max(BufferedIndexInput.bufferSize(context), DEFAULT_BUFFERSIZE));
}
protected static class WindowsIndexInput extends BufferedIndexInput {

View File

@ -159,9 +159,7 @@ public class TestNRTManager extends LuceneTestCase {
System.out.println("TEST: wrap NRTCachingDir");
}
NRTCachingDirectory nrtDir = new NRTCachingDirectory(dir, 5.0, 60.0);
conf.setMergeScheduler(nrtDir.getMergeScheduler());
dir = nrtDir;
dir = new NRTCachingDirectory(dir, 5.0, 60.0);
}
final IndexWriter writer = new IndexWriter(dir, conf);

View File

@ -40,6 +40,7 @@ import org.apache.lucene.index.codecs.CodecProvider;
import org.apache.lucene.index.codecs.SegmentInfosReader;
import org.apache.lucene.index.codecs.SegmentInfosWriter;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.MockDirectoryWrapper;
import org.apache.lucene.store.RAMDirectory;
@ -124,8 +125,8 @@ public class TestAppendingCodec extends LuceneTestCase {
}
@Override
public IndexOutput createOutput(String name) throws IOException {
return new AppendingIndexOutputWrapper(super.createOutput(name));
public IndexOutput createOutput(String name, IOContext context) throws IOException {
return new AppendingIndexOutputWrapper(super.createOutput(name, context));
}
}

View File

@ -44,7 +44,6 @@ public class TestNRTCachingDirectory extends LuceneTestCase {
Directory dir = newDirectory();
NRTCachingDirectory cachedDir = new NRTCachingDirectory(dir, 2.0, 25.0);
IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random));
conf.setMergeScheduler(cachedDir.getMergeScheduler());
RandomIndexWriter w = new RandomIndexWriter(random, cachedDir, conf);
w.w.setInfoStream(VERBOSE ? System.out : null);
final LineFileDocs docs = new LineFileDocs(random);
@ -108,13 +107,12 @@ public class TestNRTCachingDirectory extends LuceneTestCase {
Directory fsDir = FSDirectory.open(new File("/path/to/index"));
NRTCachingDirectory cachedFSDir = new NRTCachingDirectory(fsDir, 2.0, 25.0);
IndexWriterConfig conf = new IndexWriterConfig(Version.LUCENE_32, analyzer);
conf.setMergeScheduler(cachedFSDir.getMergeScheduler());
IndexWriter writer = new IndexWriter(cachedFSDir, conf);
}
public void testDeleteFile() throws Exception {
Directory dir = new NRTCachingDirectory(newDirectory(), 2.0, 25.0);
dir.createOutput("foo.txt").close();
dir.createOutput("foo.txt", IOContext.DEFAULT).close();
dir.deleteFile("foo.txt");
assertEquals(0, dir.listAll().length);
dir.close();

View File

@ -32,6 +32,7 @@ import org.apache.lucene.search.DocIdSet;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.QueryWrapperFilter;
import org.apache.lucene.store.IOContext;
/* Tracks the stream of {@link BufferedDeletes}.
* When DocumentsWriterPerThread flushes, its buffered
@ -224,7 +225,7 @@ class BufferedDeletesStream {
// Lock order: IW -> BD -> RP
assert readerPool.infoIsLive(info);
final SegmentReader reader = readerPool.get(info, false);
final SegmentReader reader = readerPool.get(info, false, IOContext.READ);
int delCount = 0;
final boolean segAllDeletes;
try {
@ -273,7 +274,7 @@ class BufferedDeletesStream {
if (coalescedDeletes != null) {
// Lock order: IW -> BD -> RP
assert readerPool.infoIsLive(info);
SegmentReader reader = readerPool.get(info, false);
SegmentReader reader = readerPool.get(info, false, IOContext.READ);
int delCount = 0;
final boolean segAllDeletes;
try {

View File

@ -17,6 +17,13 @@ package org.apache.lucene.index;
* limitations under the License.
*/
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.store.FSDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import java.io.File;
import java.io.IOException;
import java.io.PrintStream;
@ -34,12 +41,6 @@ import org.apache.lucene.index.codecs.DefaultSegmentInfosWriter;
import org.apache.lucene.index.codecs.PerDocValues;
import org.apache.lucene.index.values.IndexDocValues;
import org.apache.lucene.index.values.ValuesEnum;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FSDirectory;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.StringHelper;
@ -364,7 +365,7 @@ public class CheckIndex {
final String segmentsFileName = sis.getCurrentSegmentFileName();
IndexInput input = null;
try {
input = dir.openInput(segmentsFileName);
input = dir.openInput(segmentsFileName, IOContext.DEFAULT);
} catch (Throwable t) {
msg("ERROR: could not open segments file in directory");
if (infoStream != null)
@ -513,7 +514,7 @@ public class CheckIndex {
}
if (infoStream != null)
infoStream.print(" test: open reader.........");
reader = SegmentReader.get(true, info, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR);
reader = SegmentReader.get(true, info, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, IOContext.DEFAULT);
segInfoStat.openReaderPassed = true;

View File

@ -32,6 +32,7 @@ import java.util.concurrent.ConcurrentHashMap;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.FieldSelector;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.Lock;
import org.apache.lucene.store.LockObtainFailedException;
import org.apache.lucene.index.codecs.CodecProvider;
@ -121,7 +122,7 @@ class DirectoryReader extends IndexReader implements Cloneable {
for (int i = sis.size()-1; i >= 0; i--) {
boolean success = false;
try {
readers[i] = SegmentReader.get(readOnly, sis.info(i), termInfosIndexDivisor);
readers[i] = SegmentReader.get(readOnly, sis.info(i), termInfosIndexDivisor, IOContext.READ);
readers[i].readerFinishedListeners = readerFinishedListeners;
success = true;
} finally {
@ -170,7 +171,8 @@ class DirectoryReader extends IndexReader implements Cloneable {
try {
final SegmentInfo info = infos.info(i);
assert info.dir == dir;
final SegmentReader reader = writer.readerPool.getReadOnlyClone(info, true, termInfosIndexDivisor);
final SegmentReader reader = writer.readerPool.getReadOnlyClone(info, true, termInfosIndexDivisor,
IOContext.READ);
if (reader.numDocs() > 0 || writer.getKeepFullyDeletedSegments()) {
reader.readerFinishedListeners = readerFinishedListeners;
readers.add(reader);
@ -254,7 +256,7 @@ class DirectoryReader extends IndexReader implements Cloneable {
assert !doClone;
// this is a new reader; in case we hit an exception we can close it safely
newReader = SegmentReader.get(readOnly, infos.info(i), termInfosIndexDivisor);
newReader = SegmentReader.get(readOnly, infos.info(i), termInfosIndexDivisor, IOContext.READ);
newReader.readerFinishedListeners = readerFinishedListeners;
} else {
newReader = newReaders[i].reopenSegment(infos.info(i), doClone, readOnly);

View File

@ -30,6 +30,8 @@ import org.apache.lucene.document.Document;
import org.apache.lucene.index.DocumentsWriterDeleteQueue.DeleteSlice;
import org.apache.lucene.search.SimilarityProvider;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FlushInfo;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.BitVector;
import org.apache.lucene.util.ByteBlockPool.Allocator;
import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator;
@ -428,7 +430,7 @@ public class DocumentsWriterPerThread {
assert deleteSlice == null : "all deletes must be applied in prepareFlush";
flushState = new SegmentWriteState(infoStream, directory, segment, fieldInfos,
numDocsInRAM, writer.getConfig().getTermIndexInterval(),
fieldInfos.buildSegmentCodecs(true), pendingDeletes);
fieldInfos.buildSegmentCodecs(true), pendingDeletes, new IOContext(new FlushInfo(numDocsInRAM, bytesUsed())));
final double startMBUsed = parent.flushControl.netBytes() / 1024. / 1024.;
// Apply delete-by-docID now (delete-byDocID only
// happens when an exception is hit processing that
@ -543,7 +545,7 @@ public class DocumentsWriterPerThread {
PerDocWriteState newPerDocWriteState(int codecId) {
assert segment != null;
return new PerDocWriteState(infoStream, directory, segment, fieldInfos, bytesUsed, codecId);
return new PerDocWriteState(infoStream, directory, segment, fieldInfos, bytesUsed, codecId, IOContext.DEFAULT);
}
void setInfoStream(PrintStream infoStream) {

View File

@ -33,6 +33,7 @@ import org.apache.lucene.index.SegmentCodecs.SegmentCodecsBuilder;
import org.apache.lucene.index.codecs.CodecProvider;
import org.apache.lucene.index.values.ValueType;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.CodecUtil;
@ -270,7 +271,7 @@ public final class FieldInfos implements Iterable<FieldInfo> {
*/
public FieldInfos(Directory d, String name) throws IOException {
this((FieldNumberBiMap)null, null); // use null here to make this FIs Read-Only
final IndexInput input = d.openInput(name);
final IndexInput input = d.openInput(name, IOContext.READONCE);
try {
read(input, name);
} finally {
@ -562,7 +563,7 @@ public final class FieldInfos implements Iterable<FieldInfo> {
}
public void write(Directory d, String name) throws IOException {
IndexOutput output = d.createOutput(name);
IndexOutput output = d.createOutput(name, IOContext.READONCE);
try {
write(output);
} finally {

View File

@ -28,6 +28,7 @@ import org.apache.lucene.document.NumericField;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.BufferedIndexInput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.CloseableThreadLocal;
import org.apache.lucene.util.IOUtils;
@ -84,7 +85,7 @@ public final class FieldsReader implements Cloneable, Closeable {
/** Verifies that the code version which wrote the segment is supported. */
public static void checkCodeVersion(Directory dir, String segment) throws IOException {
final String indexStreamFN = IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELDS_INDEX_EXTENSION);
IndexInput idxStream = dir.openInput(indexStreamFN, 1024);
IndexInput idxStream = dir.openInput(indexStreamFN, IOContext.DEFAULT);
try {
int format = idxStream.readInt();
@ -113,18 +114,18 @@ public final class FieldsReader implements Cloneable, Closeable {
}
public FieldsReader(Directory d, String segment, FieldInfos fn) throws IOException {
this(d, segment, fn, BufferedIndexInput.BUFFER_SIZE, -1, 0);
this(d, segment, fn, IOContext.DEFAULT, -1, 0);
}
public FieldsReader(Directory d, String segment, FieldInfos fn, int readBufferSize, int docStoreOffset, int size) throws IOException {
public FieldsReader(Directory d, String segment, FieldInfos fn, IOContext context, int docStoreOffset, int size) throws IOException {
boolean success = false;
isOriginal = true;
try {
fieldInfos = fn;
cloneableFieldsStream = d.openInput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELDS_EXTENSION), readBufferSize);
cloneableFieldsStream = d.openInput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELDS_EXTENSION), context);
final String indexStreamFN = IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELDS_INDEX_EXTENSION);
cloneableIndexStream = d.openInput(indexStreamFN, readBufferSize);
cloneableIndexStream = d.openInput(indexStreamFN, context);
format = cloneableIndexStream.readInt();

View File

@ -23,6 +23,7 @@ import org.apache.lucene.document.Document;
import org.apache.lucene.document.Fieldable;
import org.apache.lucene.document.NumericField;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.IOUtils;
@ -65,14 +66,14 @@ final class FieldsWriter {
private IndexOutput fieldsStream;
private IndexOutput indexStream;
FieldsWriter(Directory directory, String segment) throws IOException {
FieldsWriter(Directory directory, String segment, IOContext context) throws IOException {
this.directory = directory;
this.segment = segment;
boolean success = false;
try {
fieldsStream = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELDS_EXTENSION));
indexStream = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELDS_INDEX_EXTENSION));
fieldsStream = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELDS_EXTENSION), context);
indexStream = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELDS_INDEX_EXTENSION), context);
fieldsStream.writeInt(FORMAT_CURRENT);
indexStream.writeInt(FORMAT_CURRENT);

View File

@ -1436,13 +1436,14 @@ public abstract class IndexReader implements Cloneable,Closeable {
Directory dir = null;
CompoundFileDirectory cfr = null;
IOContext context = IOContext.READ;
try {
File file = new File(filename);
String dirname = file.getAbsoluteFile().getParent();
filename = file.getName();
dir = FSDirectory.open(new File(dirname));
cfr = dir.openCompoundInput(filename, BufferedIndexInput.BUFFER_SIZE);
cfr = dir.openCompoundInput(filename, IOContext.DEFAULT);
String [] files = cfr.listAll();
ArrayUtil.mergeSort(files); // sort the array of filename so that the output is more readable
@ -1452,7 +1453,7 @@ public abstract class IndexReader implements Cloneable,Closeable {
if (extract) {
System.out.println("extract " + files[i] + " with " + len + " bytes to local directory...");
IndexInput ii = cfr.openInput(files[i]);
IndexInput ii = cfr.openInput(files[i], context);
FileOutputStream f = new FileOutputStream(files[i]);

View File

@ -45,11 +45,13 @@ import org.apache.lucene.index.SegmentCodecs.SegmentCodecsBuilder;
import org.apache.lucene.index.codecs.CodecProvider;
import org.apache.lucene.search.Query;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.BufferedIndexInput;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FlushInfo;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.Lock;
import org.apache.lucene.store.LockObtainFailedException;
import org.apache.lucene.store.MergeInfo;
import org.apache.lucene.util.BitVector;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.Constants;
@ -207,15 +209,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
* #setInfoStream}).
*/
public final static int MAX_TERM_LENGTH = DocumentsWriterPerThread.MAX_TERM_LENGTH_UTF8;
// The normal read buffer size defaults to 1024, but
// increasing this during merging seems to yield
// performance gains. However we don't want to increase
// it too much because there are quite a few
// BufferedIndexInputs created during merging. See
// LUCENE-888 for details.
private final static int MERGE_READ_BUFFER_SIZE = 4096;
// Used for printing messages
private static final AtomicInteger MESSAGE_ID = new AtomicInteger();
private int messageID = MESSAGE_ID.getAndIncrement();
@ -594,8 +587,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
* enrolled in the pool, so you should simply close()
* it when you're done (ie, do not call release()).
*/
public synchronized SegmentReader getReadOnlyClone(SegmentInfo info, boolean doOpenStores, int termInfosIndexDivisor) throws IOException {
SegmentReader sr = get(info, doOpenStores, BufferedIndexInput.BUFFER_SIZE, termInfosIndexDivisor);
public synchronized SegmentReader getReadOnlyClone(SegmentInfo info, boolean doOpenStores, int termInfosIndexDivisor, IOContext context) throws IOException {
SegmentReader sr = get(info, doOpenStores, context, termInfosIndexDivisor);
try {
return (SegmentReader) sr.clone(true);
} finally {
@ -611,8 +604,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
* @param doOpenStores
* @throws IOException
*/
public synchronized SegmentReader get(SegmentInfo info, boolean doOpenStores) throws IOException {
return get(info, doOpenStores, BufferedIndexInput.BUFFER_SIZE, config.getReaderTermsIndexDivisor());
public synchronized SegmentReader get(SegmentInfo info, boolean doOpenStores, IOContext context) throws IOException {
return get(info, doOpenStores, context, config.getReaderTermsIndexDivisor());
}
/**
@ -626,18 +619,20 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
* @param termsIndexDivisor
* @throws IOException
*/
public synchronized SegmentReader get(SegmentInfo info, boolean doOpenStores, int readBufferSize, int termsIndexDivisor) throws IOException {
public synchronized SegmentReader get(SegmentInfo info, boolean doOpenStores, IOContext context, int termsIndexDivisor) throws IOException {
if (poolReaders) {
readBufferSize = BufferedIndexInput.BUFFER_SIZE;
}
// if (poolReaders) {
// readBufferSize = BufferedIndexInput.BUFFER_SIZE;
// }
// TODO: context should be part of the key used to cache that reader in the pool.
SegmentReader sr = readerMap.get(info);
if (sr == null) {
// TODO: we may want to avoid doing this while
// synchronized
// Returns a ref, which we xfer to readerMap:
sr = SegmentReader.get(false, info.dir, info, readBufferSize, doOpenStores, termsIndexDivisor);
sr = SegmentReader.get(false, info.dir, info, doOpenStores, termsIndexDivisor, context);
sr.readerFinishedListeners = readerFinishedListeners;
if (info.dir == directory) {
@ -2185,6 +2180,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
SegmentInfo newSegment = flushedSegment.segmentInfo;
setDiagnostics(newSegment, "flush");
IOContext context = new IOContext(new FlushInfo(newSegment.docCount, newSegment.sizeInBytes(true)));
boolean success = false;
try {
@ -2192,11 +2189,11 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
String compoundFileName = IndexFileNames.segmentFileName(newSegment.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION);
message("creating compound file " + compoundFileName);
// Now build compound file
final Directory cfsDir = directory.createCompoundOutput(compoundFileName);
final Directory cfsDir = directory.createCompoundOutput(compoundFileName, context);
IOException prior = null;
try {
for(String fileName : newSegment.files()) {
directory.copy(cfsDir, fileName, fileName);
directory.copy(cfsDir, fileName, fileName, context);
}
} catch(IOException ex) {
prior = ex;
@ -2230,7 +2227,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
// shortly-to-be-opened SegmentReader and let it
// carry the changes; there's no reason to use
// filesystem as intermediary here.
flushedSegment.liveDocs.write(directory, delFileName);
flushedSegment.liveDocs.write(directory, delFileName, context);
success2 = true;
} finally {
if (!success2) {
@ -2399,11 +2396,13 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
// optimize case only for segments that don't share doc stores
&& versionComparator.compare(info.getVersion(), "3.1") >= 0;
}
IOContext context = new IOContext(new MergeInfo(info.docCount, info.sizeInBytes(true), true, false));
if (createCFS) {
copySegmentIntoCFS(info, newSegName);
copySegmentIntoCFS(info, newSegName, context);
} else {
copySegmentAsIs(info, newSegName, dsNames, dsFilesCopied);
copySegmentAsIs(info, newSegName, dsNames, dsFilesCopied, context);
}
infos.add(info);
@ -2447,6 +2446,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
*/
public void addIndexes(IndexReader... readers) throws CorruptIndexException, IOException {
ensureOpen();
int numDocs = 0;
try {
if (infoStream != null)
@ -2454,15 +2454,19 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
flush(false, true);
String mergedName = newSegmentName();
for (IndexReader indexReader : readers) {
numDocs += indexReader.numDocs();
}
final IOContext context = new IOContext(new MergeInfo(numDocs, -1, true, false));
// TODO: somehow we should fix this merge so it's
// abortable so that IW.close(false) is able to stop it
SegmentMerger merger = new SegmentMerger(directory, config.getTermIndexInterval(),
mergedName, null, payloadProcessorProvider,
globalFieldNumberMap.newFieldInfos(SegmentCodecsBuilder.create(codecs)));
globalFieldNumberMap.newFieldInfos(SegmentCodecsBuilder.create(codecs)), context);
for (IndexReader reader : readers) // add new indexes
merger.add(reader);
int docCount = merger.merge(); // merge 'em
final FieldInfos fieldInfos = merger.fieldInfos();
@ -2483,7 +2487,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
// Now create the compound file if needed
if (useCompoundFile) {
merger.createCompoundFile(mergedName + ".cfs", info);
merger.createCompoundFile(mergedName + ".cfs", info, context);
// delete new non cfs files directly: they were never
// registered with IFD
@ -2507,19 +2511,19 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
}
/** Copies the segment into the IndexWriter's directory, as a compound segment. */
private void copySegmentIntoCFS(SegmentInfo info, String segName) throws IOException {
private void copySegmentIntoCFS(SegmentInfo info, String segName, IOContext context) throws IOException {
String segFileName = IndexFileNames.segmentFileName(segName, "", IndexFileNames.COMPOUND_FILE_EXTENSION);
Collection<String> files = info.files();
final CompoundFileDirectory cfsdir = directory.createCompoundOutput(segFileName);
final CompoundFileDirectory cfsdir = directory.createCompoundOutput(segFileName, context);
try {
for (String file : files) {
String newFileName = segName + IndexFileNames.stripSegmentName(file);
if (!IndexFileNames.matchesExtension(file, IndexFileNames.DELETES_EXTENSION)
&& !IndexFileNames.isSeparateNormsFile(file)) {
info.dir.copy(cfsdir, file, file);
info.dir.copy(cfsdir, file, file, context);
} else {
assert !directory.fileExists(newFileName): "file \"" + newFileName + "\" already exists";
info.dir.copy(directory, file, newFileName);
info.dir.copy(directory, file, newFileName, context);
}
}
} finally {
@ -2533,7 +2537,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
/** Copies the segment files as-is into the IndexWriter's directory. */
private void copySegmentAsIs(SegmentInfo info, String segName,
Map<String, String> dsNames, Set<String> dsFilesCopied)
Map<String, String> dsNames, Set<String> dsFilesCopied, IOContext context)
throws IOException {
// Determine if the doc store of this segment needs to be copied. It's
// only relevant for segments that share doc store with others,
@ -2569,7 +2573,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
}
assert !directory.fileExists(newFileName): "file \"" + newFileName + "\" already exists";
info.dir.copy(directory, file, newFileName);
info.dir.copy(directory, file, newFileName, context);
}
info.setDocStore(info.getDocStoreOffset(), newDsName, info.getDocStoreIsCompoundFile());
@ -3425,9 +3429,11 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
int mergedDocCount = 0;
List<SegmentInfo> sourceSegments = merge.segments;
IOContext context = new IOContext(merge.getMergeInfo());
SegmentMerger merger = new SegmentMerger(directory, config.getTermIndexInterval(), mergedName, merge,
payloadProcessorProvider, merge.info.getFieldInfos());
payloadProcessorProvider, merge.info.getFieldInfos(), context);
if (infoStream != null) {
message("merging " + merge.segString(directory) + " mergeVectors=" + merge.info.getFieldInfos().hasVectors());
@ -3448,7 +3454,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
// Hold onto the "live" reader; we will use this to
// commit merged deletes
final SegmentReader reader = readerPool.get(info, true,
MERGE_READ_BUFFER_SIZE,
context,
-config.getReaderTermsIndexDivisor());
merge.readers.add(reader);
@ -3502,7 +3508,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
if (infoStream != null) {
message("create compound file " + compoundFileName);
}
merger.createCompoundFile(compoundFileName, merge.info);
merger.createCompoundFile(compoundFileName, merge.info, new IOContext(merge.getMergeInfo()));
success = true;
} catch (IOException ioe) {
synchronized(this) {
@ -3574,7 +3580,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
// keep deletes (it's costly to open entire reader
// when we just need deletes)
final SegmentReader mergedReader = readerPool.get(merge.info, loadDocStores, BufferedIndexInput.BUFFER_SIZE, termsIndexDivisor);
final SegmentReader mergedReader = readerPool.get(merge.info, loadDocStores, context, termsIndexDivisor);
try {
if (poolReaders && mergedSegmentWarmer != null) {
mergedSegmentWarmer.warm(mergedReader);

View File

@ -18,6 +18,7 @@ package org.apache.lucene.index;
*/
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.MergeInfo;
import org.apache.lucene.util.SetOnce;
import org.apache.lucene.util.SetOnce.AlreadySetException;
@ -189,6 +190,10 @@ public abstract class MergePolicy implements java.io.Closeable {
}
return total;
}
public MergeInfo getMergeInfo() {
return new MergeInfo(totalDocCount, estimatedMergeBytes, isExternal, optimize);
}
}
/**

View File

@ -22,6 +22,7 @@ import java.util.Collection;
import java.util.Map;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.IOContext.Context;
import org.apache.lucene.util.IOUtils;
// TODO FI: norms could actually be stored as doc store
@ -49,7 +50,7 @@ final class NormsWriter extends InvertedDocEndConsumer {
}
final String normsFileName = IndexFileNames.segmentFileName(state.segmentName, "", IndexFileNames.NORMS_EXTENSION);
IndexOutput normsOut = state.directory.createOutput(normsFileName);
IndexOutput normsOut = state.directory.createOutput(normsFileName, state.context);
boolean success = false;
try {
normsOut.writeBytes(SegmentNorms.NORMS_HEADER, 0, SegmentNorms.NORMS_HEADER.length);

View File

@ -20,6 +20,7 @@ import java.util.concurrent.atomic.AtomicLong;
import org.apache.lucene.index.codecs.PerDocConsumer;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
/**
* Encapsulates all necessary state to initiate a {@link PerDocConsumer} and
@ -35,10 +36,11 @@ public class PerDocWriteState {
public final AtomicLong bytesUsed;
public final SegmentCodecs segmentCodecs;
public final int codecId;
public final IOContext context;
PerDocWriteState(PrintStream infoStream, Directory directory,
String segmentName, FieldInfos fieldInfos, AtomicLong bytesUsed,
int codecId) {
int codecId, IOContext context) {
this.infoStream = infoStream;
this.directory = directory;
this.segmentName = segmentName;
@ -46,6 +48,7 @@ public class PerDocWriteState {
this.segmentCodecs = fieldInfos.buildSegmentCodecs(false);
this.codecId = codecId;
this.bytesUsed = bytesUsed;
this.context = context;
}
PerDocWriteState(SegmentWriteState state) {
@ -56,6 +59,7 @@ public class PerDocWriteState {
fieldInfos = state.fieldInfos;
codecId = state.codecId;
bytesUsed = new AtomicLong(0);
context = state.context;
}
PerDocWriteState(PerDocWriteState state, int codecId) {
@ -66,5 +70,6 @@ public class PerDocWriteState {
this.segmentCodecs = state.segmentCodecs;
this.codecId = codecId;
this.bytesUsed = state.bytesUsed;
this.context = state.context;
}
}

View File

@ -36,6 +36,7 @@ import org.apache.lucene.index.codecs.TermsConsumer;
import org.apache.lucene.index.codecs.DocValuesConsumer;
import org.apache.lucene.index.values.IndexDocValues;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.IOUtils;
/**
@ -99,7 +100,7 @@ final class PerFieldCodecWrapper extends Codec {
private final Map<String, FieldsProducer> codecs = new HashMap<String, FieldsProducer>();
public FieldsReader(Directory dir, FieldInfos fieldInfos, SegmentInfo si,
int readBufferSize, int indexDivisor) throws IOException {
IOContext context, int indexDivisor) throws IOException {
final Map<Codec, FieldsProducer> producers = new HashMap<Codec, FieldsProducer>();
boolean success = false;
@ -111,7 +112,7 @@ final class PerFieldCodecWrapper extends Codec {
Codec codec = segmentCodecs.codecs[fi.getCodecId()];
if (!producers.containsKey(codec)) {
producers.put(codec, codec.fieldsProducer(new SegmentReadState(dir,
si, fieldInfos, readBufferSize, indexDivisor, fi.getCodecId())));
si, fieldInfos, context, indexDivisor, fi.getCodecId())));
}
codecs.put(fi.name, producers.get(codec));
}
@ -187,7 +188,7 @@ final class PerFieldCodecWrapper extends Codec {
public FieldsProducer fieldsProducer(SegmentReadState state)
throws IOException {
return new FieldsReader(state.dir, state.fieldInfos, state.segmentInfo,
state.readBufferSize, state.termsIndexDivisor);
state.context, state.termsIndexDivisor);
}
@Override
@ -212,14 +213,14 @@ final class PerFieldCodecWrapper extends Codec {
@Override
public PerDocValues docsProducer(SegmentReadState state) throws IOException {
return new PerDocProducers(state.dir, state.fieldInfos, state.segmentInfo,
state.readBufferSize, state.termsIndexDivisor);
state.context, state.termsIndexDivisor);
}
private final class PerDocProducers extends PerDocValues {
private final TreeMap<String, PerDocValues> codecs = new TreeMap<String, PerDocValues>();
public PerDocProducers(Directory dir, FieldInfos fieldInfos, SegmentInfo si,
int readBufferSize, int indexDivisor) throws IOException {
IOContext context, int indexDivisor) throws IOException {
final Map<Codec, PerDocValues> producers = new HashMap<Codec, PerDocValues>();
boolean success = false;
try {
@ -229,7 +230,7 @@ final class PerFieldCodecWrapper extends Codec {
Codec codec = segmentCodecs.codecs[fi.getCodecId()];
if (!producers.containsKey(codec)) {
producers.put(codec, codec.docsProducer(new SegmentReadState(dir,
si, fieldInfos, readBufferSize, indexDivisor, fi.getCodecId())));
si, fieldInfos, context, indexDivisor, fi.getCodecId())));
}
codecs.put(fi.name, producers.get(codec));
}

View File

@ -25,6 +25,7 @@ import org.apache.lucene.index.codecs.FieldsProducer;
import org.apache.lucene.index.codecs.PerDocValues;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.IOUtils;
/** Holds core readers that are shared (unchanged) when
@ -47,7 +48,7 @@ final class SegmentCoreReaders {
final Directory dir;
final Directory cfsDir;
final int readBufferSize;
final IOContext context;
final int termsIndexDivisor;
private final SegmentReader owner;
@ -59,7 +60,7 @@ final class SegmentCoreReaders {
SegmentCoreReaders(SegmentReader owner, Directory dir, SegmentInfo si, int readBufferSize, int termsIndexDivisor) throws IOException {
SegmentCoreReaders(SegmentReader owner, Directory dir, SegmentInfo si, IOContext context, int termsIndexDivisor) throws IOException {
if (termsIndexDivisor == 0) {
throw new IllegalArgumentException("indexDivisor must be < 0 (don't load terms index) or greater than 0 (got 0)");
@ -67,7 +68,7 @@ final class SegmentCoreReaders {
segment = si.name;
final SegmentCodecs segmentCodecs = si.getSegmentCodecs();
this.readBufferSize = readBufferSize;
this.context = context;
this.dir = dir;
boolean success = false;
@ -75,7 +76,7 @@ final class SegmentCoreReaders {
try {
Directory dir0 = dir;
if (si.getUseCompoundFile()) {
cfsReader = dir.openCompoundInput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.COMPOUND_FILE_EXTENSION), readBufferSize);
cfsReader = dir.openCompoundInput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.COMPOUND_FILE_EXTENSION), context);
dir0 = cfsReader;
}
cfsDir = dir0;
@ -84,7 +85,7 @@ final class SegmentCoreReaders {
this.termsIndexDivisor = termsIndexDivisor;
final Codec codec = segmentCodecs.codec();
final SegmentReadState segmentReadState = new SegmentReadState(cfsDir, si, fieldInfos, readBufferSize, termsIndexDivisor);
final SegmentReadState segmentReadState = new SegmentReadState(cfsDir, si, fieldInfos, context, termsIndexDivisor);
// Ask codec for its Fields
fields = codec.fieldsProducer(segmentReadState);
assert fields != null;
@ -141,7 +142,7 @@ final class SegmentCoreReaders {
assert storeCFSReader == null;
storeCFSReader = dir.openCompoundInput(
IndexFileNames.segmentFileName(si.getDocStoreSegment(), "", IndexFileNames.COMPOUND_FILE_STORE_EXTENSION),
readBufferSize);
context);
storeDir = storeCFSReader;
assert storeDir != null;
} else {
@ -153,7 +154,7 @@ final class SegmentCoreReaders {
// was not used, but then we are asked to open doc
// stores after the segment has switched to CFS
if (cfsReader == null) {
cfsReader = dir.openCompoundInput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.COMPOUND_FILE_EXTENSION), readBufferSize);
cfsReader = dir.openCompoundInput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.COMPOUND_FILE_EXTENSION), context);
}
storeDir = cfsReader;
assert storeDir != null;
@ -163,7 +164,7 @@ final class SegmentCoreReaders {
}
final String storesSegment = si.getDocStoreSegment();
fieldsReaderOrig = new FieldsReader(storeDir, storesSegment, fieldInfos, readBufferSize,
fieldsReaderOrig = new FieldsReader(storeDir, storesSegment, fieldInfos, context,
si.getDocStoreOffset(), si.docCount);
// Verify two sources of "maxDoc" agree:
@ -172,7 +173,7 @@ final class SegmentCoreReaders {
}
if (si.getHasVectors()) { // open term vector files only as needed
termVectorsReaderOrig = new TermVectorsReader(storeDir, storesSegment, fieldInfos, readBufferSize, si.getDocStoreOffset(), si.docCount);
termVectorsReaderOrig = new TermVectorsReader(storeDir, storesSegment, fieldInfos, context, si.getDocStoreOffset(), si.docCount);
}
}
}

View File

@ -30,8 +30,8 @@ import java.util.Set;
import org.apache.lucene.index.codecs.Codec;
import org.apache.lucene.index.codecs.CodecProvider;
import org.apache.lucene.index.codecs.DefaultSegmentInfosWriter;
import org.apache.lucene.store.BufferedIndexInput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.Constants;
@ -247,7 +247,7 @@ public final class SegmentInfo implements Cloneable {
}
final Directory dirToTest;
if (isCompoundFile) {
dirToTest = dir.openCompoundInput(IndexFileNames.segmentFileName(storesSegment, "", ext), BufferedIndexInput.BUFFER_SIZE);
dirToTest = dir.openCompoundInput(IndexFileNames.segmentFileName(storesSegment, "", ext), IOContext.READONCE);
} else {
dirToTest = dir;
}
@ -266,7 +266,7 @@ public final class SegmentInfo implements Cloneable {
Directory dir0 = dir;
if (isCompoundFile && checkCompoundFile) {
dir0 = dir.openCompoundInput(IndexFileNames.segmentFileName(name,
"", IndexFileNames.COMPOUND_FILE_EXTENSION), BufferedIndexInput.BUFFER_SIZE);
"", IndexFileNames.COMPOUND_FILE_EXTENSION), IOContext.READONCE);
}
try {
fieldInfos = new FieldInfos(dir0, IndexFileNames.segmentFileName(name,

View File

@ -37,6 +37,7 @@ import org.apache.lucene.index.codecs.DefaultSegmentInfosWriter;
import org.apache.lucene.index.codecs.SegmentInfosReader;
import org.apache.lucene.index.codecs.SegmentInfosWriter;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.NoSuchDirectoryException;
@ -254,7 +255,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfo> {
try {
SegmentInfosReader infosReader = codecs.getSegmentInfosReader();
infosReader.read(directory, segmentFileName, codecs, this);
infosReader.read(directory, segmentFileName, codecs, this, IOContext.READ);
success = true;
}
finally {
@ -322,7 +323,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfo> {
try {
SegmentInfosWriter infosWriter = codecs.getSegmentInfosWriter();
segnOutput = infosWriter.writeInfos(directory, segmentFileName, this);
segnOutput = infosWriter.writeInfos(directory, segmentFileName, this, IOContext.DEFAULT);
infosWriter.prepareCommit(segnOutput);
pendingSegnOutput = segnOutput;
success = true;
@ -597,7 +598,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfo> {
for(int i=0;i<defaultGenFileRetryCount;i++) {
IndexInput genInput = null;
try {
genInput = directory.openInput(IndexFileNames.SEGMENTS_GEN);
genInput = directory.openInput(IndexFileNames.SEGMENTS_GEN, IOContext.READONCE);
} catch (FileNotFoundException e) {
if (infoStream != null) {
message("segments.gen open: FileNotFoundException " + e);
@ -814,7 +815,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfo> {
}
private final long writeGlobalFieldMap(FieldNumberBiMap map, Directory dir, String name) throws IOException {
final IndexOutput output = dir.createOutput(name);
final IndexOutput output = dir.createOutput(name, IOContext.READONCE);
boolean success = false;
long version;
try {
@ -843,7 +844,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfo> {
private void readGlobalFieldMap(FieldNumberBiMap map, Directory dir) throws IOException {
final String name = getGlobalFieldNumberName(lastGlobalFieldMapVersion);
final IndexInput input = dir.openInput(name);
final IndexInput input = dir.openInput(name, IOContext.READONCE);
try {
map.read(input);
} finally {
@ -934,7 +935,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfo> {
}
try {
IndexOutput genOutput = dir.createOutput(IndexFileNames.SEGMENTS_GEN);
IndexOutput genOutput = dir.createOutput(IndexFileNames.SEGMENTS_GEN, IOContext.READONCE);
try {
genOutput.writeInt(FORMAT_SEGMENTS_GEN_CURRENT);
genOutput.writeLong(generation);

View File

@ -33,6 +33,7 @@ import org.apache.lucene.index.codecs.PerDocConsumer;
import org.apache.lucene.index.codecs.PerDocValues;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.Bits;
@ -68,8 +69,10 @@ final class SegmentMerger {
private SegmentWriteState segmentWriteState;
private PayloadProcessorProvider payloadProcessorProvider;
private IOContext context;
SegmentMerger(Directory dir, int termIndexInterval, String name, MergePolicy.OneMerge merge, PayloadProcessorProvider payloadProcessorProvider, FieldInfos fieldInfos) {
SegmentMerger(Directory dir, int termIndexInterval, String name, MergePolicy.OneMerge merge, PayloadProcessorProvider payloadProcessorProvider, FieldInfos fieldInfos, IOContext context) {
this.payloadProcessorProvider = payloadProcessorProvider;
directory = dir;
segment = name;
@ -85,6 +88,7 @@ final class SegmentMerger {
};
}
this.termIndexInterval = termIndexInterval;
this.context = context;
}
public FieldInfos fieldInfos() {
@ -129,19 +133,19 @@ final class SegmentMerger {
* deletion files, this SegmentInfo must not reference such files when this
* method is called, because they are not allowed within a compound file.
*/
final Collection<String> createCompoundFile(String fileName, final SegmentInfo info)
final Collection<String> createCompoundFile(String fileName, final SegmentInfo info, IOContext context)
throws IOException {
// Now merge all added files
Collection<String> files = info.files();
CompoundFileDirectory cfsDir = directory.createCompoundOutput(fileName);
CompoundFileDirectory cfsDir = directory.createCompoundOutput(fileName, context);
try {
for (String file : files) {
assert !IndexFileNames.matchesExtension(file, IndexFileNames.DELETES_EXTENSION)
: ".del file is not allowed in .cfs: " + file;
assert !IndexFileNames.isSeparateNormsFile(file)
: "separate norms file (.s[0-9]+) is not allowed in .cfs: " + file;
directory.copy(cfsDir, file, file);
directory.copy(cfsDir, file, file, context);
checkAbort.work(directory.fileLength(file));
}
} finally {
@ -236,9 +240,7 @@ final class SegmentMerger {
int docCount = 0;
setMatchingSegmentReaders();
final FieldsWriter fieldsWriter = new FieldsWriter(directory, segment);
final FieldsWriter fieldsWriter = new FieldsWriter(directory, segment, context);
try {
int idx = 0;
for (IndexReader reader : readers) {
@ -272,8 +274,7 @@ final class SegmentMerger {
// entering the index. See LUCENE-1282 for
// details.
throw new RuntimeException("mergeFields produced an invalid result: docCount is " + docCount + " but fdx file size is " + fdxFileLength + " file=" + fileName + " file exists?=" + directory.fileExists(fileName) + "; now aborting this merge to prevent index corruption");
segmentWriteState = new SegmentWriteState(null, directory, segment, fieldInfos, docCount, termIndexInterval, codecInfo, null);
segmentWriteState = new SegmentWriteState(null, directory, segment, fieldInfos, docCount, termIndexInterval, codecInfo, null, context);
return docCount;
}
@ -360,7 +361,7 @@ final class SegmentMerger {
*/
private final void mergeVectors() throws IOException {
TermVectorsWriter termVectorsWriter =
new TermVectorsWriter(directory, segment, fieldInfos);
new TermVectorsWriter(directory, segment, fieldInfos, context);
try {
int idx = 0;
@ -629,7 +630,7 @@ final class SegmentMerger {
for (FieldInfo fi : fieldInfos) {
if (fi.isIndexed && !fi.omitNorms) {
if (output == null) {
output = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.NORMS_EXTENSION));
output = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.NORMS_EXTENSION), context);
output.writeBytes(SegmentNorms.NORMS_HEADER, SegmentNorms.NORMS_HEADER.length);
}
for (IndexReader reader : readers) {

View File

@ -20,9 +20,10 @@ package org.apache.lucene.index;
import java.io.IOException;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.lucene.store.FlushInfo;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
/**
* Byte[] referencing is used because a new norm object needs
* to be created for each clone, and the byte array is all
@ -219,7 +220,7 @@ final class SegmentNorms implements Cloneable {
// NOTE: norms are re-written in regular directory, not cfs
si.advanceNormGen(this.number);
final String normFileName = si.getNormFileName(this.number);
IndexOutput out = owner.directory().createOutput(normFileName);
IndexOutput out = owner.directory().createOutput(normFileName, new IOContext(new FlushInfo(si.docCount, 0)));
boolean success = false;
try {
try {

View File

@ -18,6 +18,7 @@ package org.apache.lucene.index;
*/
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
/**
* @lucene.experimental
@ -26,7 +27,7 @@ public class SegmentReadState {
public final Directory dir;
public final SegmentInfo segmentInfo;
public final FieldInfos fieldInfos;
public final int readBufferSize;
public final IOContext context;
// NOTE: if this is < 0, that means "defer terms index
// load until needed". But if the codec must load the
@ -37,20 +38,20 @@ public class SegmentReadState {
public final int codecId;
public SegmentReadState(Directory dir, SegmentInfo info,
FieldInfos fieldInfos, int readBufferSize, int termsIndexDivisor) {
this(dir, info, fieldInfos, readBufferSize, termsIndexDivisor, -1);
FieldInfos fieldInfos, IOContext context, int termsIndexDivisor) {
this(dir, info, fieldInfos, context, termsIndexDivisor, -1);
}
public SegmentReadState(Directory dir,
SegmentInfo info,
FieldInfos fieldInfos,
int readBufferSize,
IOContext context,
int termsIndexDivisor,
int codecId) {
this.dir = dir;
this.segmentInfo = info;
this.fieldInfos = fieldInfos;
this.readBufferSize = readBufferSize;
this.context = context;
this.termsIndexDivisor = termsIndexDivisor;
this.codecId = codecId;
}

View File

@ -32,6 +32,7 @@ import org.apache.lucene.document.FieldSelector;
import org.apache.lucene.index.codecs.PerDocValues;
import org.apache.lucene.store.BufferedIndexInput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.BitVector;
import org.apache.lucene.util.Bits;
@ -46,7 +47,6 @@ public class SegmentReader extends IndexReader implements Cloneable {
protected boolean readOnly;
private SegmentInfo si;
private int readBufferSize;
private final ReaderContext readerContext = new AtomicReaderContext(this);
CloseableThreadLocal<FieldsReader> fieldsReaderLocal = new FieldsReaderLocal();
CloseableThreadLocal<TermVectorsReader> termVectorsLocal = new CloseableThreadLocal<TermVectorsReader>();
@ -88,8 +88,9 @@ public class SegmentReader extends IndexReader implements Cloneable {
* @throws CorruptIndexException if the index is corrupt
* @throws IOException if there is a low-level IO error
*/
public static SegmentReader get(boolean readOnly, SegmentInfo si, int termInfosIndexDivisor) throws CorruptIndexException, IOException {
return get(readOnly, si.dir, si, BufferedIndexInput.BUFFER_SIZE, true, termInfosIndexDivisor);
public static SegmentReader get(boolean readOnly, SegmentInfo si, int termInfosIndexDivisor, IOContext context) throws CorruptIndexException, IOException {
// TODO should we check if readOnly and context combination makes sense like asserting that if read only we don't get a default?
return get(readOnly, si.dir, si, true, termInfosIndexDivisor, context);
}
/**
@ -99,25 +100,23 @@ public class SegmentReader extends IndexReader implements Cloneable {
public static SegmentReader get(boolean readOnly,
Directory dir,
SegmentInfo si,
int readBufferSize,
boolean doOpenStores,
int termInfosIndexDivisor)
int termInfosIndexDivisor,
IOContext context)
throws CorruptIndexException, IOException {
SegmentReader instance = new SegmentReader();
instance.readOnly = readOnly;
instance.si = si;
instance.readBufferSize = readBufferSize;
boolean success = false;
try {
instance.core = new SegmentCoreReaders(instance, dir, si, readBufferSize, termInfosIndexDivisor);
instance.core = new SegmentCoreReaders(instance, dir, si, context, termInfosIndexDivisor);
if (doOpenStores) {
instance.core.openDocStores(si);
}
instance.loadLiveDocs();
instance.openNorms(instance.core.cfsDir, readBufferSize);
instance.openNorms(instance.core.cfsDir, context);
success = true;
} finally {
@ -161,7 +160,7 @@ public class SegmentReader extends IndexReader implements Cloneable {
private void loadLiveDocs() throws IOException {
// NOTE: the bitvector is stored using the regular directory, not cfs
if (hasDeletions(si)) {
liveDocs = new BitVector(directory(), si.getDelFileName());
liveDocs = new BitVector(directory(), si.getDelFileName(), IOContext.DEFAULT);
if (liveDocs.getVersion() < BitVector.VERSION_DGAPS_CLEARED) {
liveDocs.invertAll();
}
@ -253,7 +252,6 @@ public class SegmentReader extends IndexReader implements Cloneable {
clone.core = core;
clone.readOnly = openReadOnly;
clone.si = si;
clone.readBufferSize = readBufferSize;
clone.pendingDeleteCount = pendingDeleteCount;
clone.readerFinishedListeners = readerFinishedListeners;
@ -298,7 +296,7 @@ public class SegmentReader extends IndexReader implements Cloneable {
// If we are not cloning, then this will open anew
// any norms that have changed:
clone.openNorms(si.getUseCompoundFile() ? core.getCFSReader() : directory(), readBufferSize);
clone.openNorms(si.getUseCompoundFile() ? core.getCFSReader() : directory(), IOContext.DEFAULT);
success = true;
} finally {
@ -340,7 +338,7 @@ public class SegmentReader extends IndexReader implements Cloneable {
final String delFileName = si.getDelFileName();
boolean success = false;
try {
liveDocs.write(directory(), delFileName);
liveDocs.write(directory(), delFileName, IOContext.DEFAULT);
success = true;
} finally {
if (!success) {
@ -580,7 +578,7 @@ public class SegmentReader extends IndexReader implements Cloneable {
norm.copyOnWrite()[doc] = value; // set the value
}
private void openNorms(Directory cfsDir, int readBufferSize) throws IOException {
private void openNorms(Directory cfsDir, IOContext context) throws IOException {
long nextNormSeek = SegmentNorms.NORMS_HEADER.length; //skip header (header unused for now)
int maxDoc = maxDoc();
for (FieldInfo fi : core.fieldInfos) {
@ -604,7 +602,7 @@ public class SegmentReader extends IndexReader implements Cloneable {
if (singleNormFile) {
normSeek = nextNormSeek;
if (singleNormStream == null) {
singleNormStream = d.openInput(fileName, readBufferSize);
singleNormStream = d.openInput(fileName, context);
singleNormRef = new AtomicInteger(1);
} else {
singleNormRef.incrementAndGet();
@ -614,7 +612,7 @@ public class SegmentReader extends IndexReader implements Cloneable {
// If this were to change in the future, a clone could be done here.
normInput = singleNormStream;
} else {
normInput = d.openInput(fileName);
normInput = d.openInput(fileName, context);
// if the segment was created in 3.2 or after, we wrote the header for sure,
// and don't need to do the sketchy file size check. otherwise, we check
// if the size is exactly equal to maxDoc to detect a headerless file.

View File

@ -20,6 +20,7 @@ package org.apache.lucene.index;
import java.io.PrintStream;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.BitVector;
/**
@ -51,9 +52,11 @@ public class SegmentWriteState {
* slower. Searching is typically not dominated by dictionary lookup, so
* tweaking this is rarely useful.*/
public int termIndexInterval; // TODO: this should be private to the codec, not settable here or in IWC
public final IOContext context;
public SegmentWriteState(PrintStream infoStream, Directory directory, String segmentName, FieldInfos fieldInfos,
int numDocs, int termIndexInterval, SegmentCodecs segmentCodecs, BufferedDeletes segDeletes) {
int numDocs, int termIndexInterval, SegmentCodecs segmentCodecs, BufferedDeletes segDeletes, IOContext context) {
this.infoStream = infoStream;
this.segDeletes = segDeletes;
this.directory = directory;
@ -63,6 +66,7 @@ public class SegmentWriteState {
this.termIndexInterval = termIndexInterval;
this.segmentCodecs = segmentCodecs;
codecId = -1;
this.context = context;
}
/**
@ -76,6 +80,7 @@ public class SegmentWriteState {
numDocs = state.numDocs;
termIndexInterval = state.termIndexInterval;
segmentCodecs = state.segmentCodecs;
context = state.context;
this.codecId = codecId;
segDeletes = state.segDeletes;
}

View File

@ -20,6 +20,7 @@ package org.apache.lucene.index;
import java.io.IOException;
import org.apache.lucene.document.Fieldable;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.RamUsageEstimator;
@ -59,7 +60,7 @@ final class StoredFieldsWriter {
// It's possible that all documents seen in this segment
// hit non-aborting exceptions, in which case we will
// not have yet init'd the FieldsWriter:
initFieldsWriter();
initFieldsWriter(state.context);
fill(state.numDocs);
}
@ -75,9 +76,9 @@ final class StoredFieldsWriter {
}
}
private synchronized void initFieldsWriter() throws IOException {
private synchronized void initFieldsWriter(IOContext context) throws IOException {
if (fieldsWriter == null) {
fieldsWriter = new FieldsWriter(docWriter.directory, docWriter.getSegment());
fieldsWriter = new FieldsWriter(docWriter.directory, docWriter.getSegment(), context);
lastDocID = 0;
}
}
@ -107,7 +108,7 @@ final class StoredFieldsWriter {
void finishDocument() throws IOException {
assert docWriter.writer.testPoint("StoredFieldsWriter.finishDocument start");
initFieldsWriter();
initFieldsWriter(IOContext.DEFAULT);
fill(docState.docID);
if (fieldsWriter != null && numStoredFields > 0) {

View File

@ -17,9 +17,12 @@ package org.apache.lucene.index;
* limitations under the License.
*/
import org.apache.lucene.index.MergePolicy.OneMerge;
import org.apache.lucene.store.BufferedIndexInput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IOContext.Context;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
@ -63,29 +66,24 @@ class TermVectorsReader implements Cloneable, Closeable {
private final int format;
TermVectorsReader(Directory d, String segment, FieldInfos fieldInfos)
TermVectorsReader(Directory d, String segment, FieldInfos fieldInfos, IOContext context)
throws CorruptIndexException, IOException {
this(d, segment, fieldInfos, BufferedIndexInput.BUFFER_SIZE);
}
TermVectorsReader(Directory d, String segment, FieldInfos fieldInfos, int readBufferSize)
throws CorruptIndexException, IOException {
this(d, segment, fieldInfos, readBufferSize, -1, 0);
this(d, segment, fieldInfos, context, -1, 0);
}
TermVectorsReader(Directory d, String segment, FieldInfos fieldInfos, int readBufferSize, int docStoreOffset, int size)
TermVectorsReader(Directory d, String segment, FieldInfos fieldInfos, IOContext context, int docStoreOffset, int size)
throws CorruptIndexException, IOException {
boolean success = false;
try {
String idxName = IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_INDEX_EXTENSION);
tvx = d.openInput(idxName, readBufferSize);
tvx = d.openInput(idxName, context);
format = checkValidFormat(tvx, idxName);
String fn = IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION);
tvd = d.openInput(fn, readBufferSize);
tvd = d.openInput(fn, context);
final int tvdFormat = checkValidFormat(tvd, fn);
fn = IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_FIELDS_EXTENSION);
tvf = d.openInput(fn, readBufferSize);
tvf = d.openInput(fn, context);
final int tvfFormat = checkValidFormat(tvf, fn);
assert format == tvdFormat;

View File

@ -20,7 +20,10 @@ package org.apache.lucene.index;
import java.io.IOException;
import java.util.Map;
import org.apache.lucene.store.FlushInfo;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.IOContext.Context;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
@ -89,13 +92,14 @@ final class TermVectorsTermsWriter extends TermsHashConsumer {
if (tvx == null) {
boolean success = false;
try {
IOContext context = new IOContext(new FlushInfo(docWriter.getNumDocsInRAM(), docWriter.bytesUsed()));
// If we hit an exception while init'ing the term
// vector output files, we must abort this segment
// because those files will be in an unknown
// state:
tvx = docWriter.directory.createOutput(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_INDEX_EXTENSION));
tvd = docWriter.directory.createOutput(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION));
tvf = docWriter.directory.createOutput(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_FIELDS_EXTENSION));
tvx = docWriter.directory.createOutput(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_INDEX_EXTENSION), context);
tvd = docWriter.directory.createOutput(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION), context);
tvf = docWriter.directory.createOutput(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_FIELDS_EXTENSION), context);
tvx.writeInt(TermVectorsReader.FORMAT_CURRENT);
tvd.writeInt(TermVectorsReader.FORMAT_CURRENT);

View File

@ -18,6 +18,7 @@ package org.apache.lucene.index;
*/
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
@ -31,15 +32,15 @@ final class TermVectorsWriter {
private FieldInfos fieldInfos;
public TermVectorsWriter(Directory directory, String segment,
FieldInfos fieldInfos) throws IOException {
FieldInfos fieldInfos, IOContext context) throws IOException {
boolean success = false;
try {
// Open files for TermVector storage
tvx = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_INDEX_EXTENSION));
tvx = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_INDEX_EXTENSION), context);
tvx.writeInt(TermVectorsReader.FORMAT_CURRENT);
tvd = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION));
tvd = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION), context);
tvd.writeInt(TermVectorsReader.FORMAT_CURRENT);
tvf = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_FIELDS_EXTENSION));
tvf = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_FIELDS_EXTENSION), context);
tvf.writeInt(TermVectorsReader.FORMAT_CURRENT);
success = true;
} finally {

View File

@ -37,6 +37,7 @@ import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.index.codecs.standard.StandardPostingsReader; // javadocs
import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Bits;
@ -107,7 +108,7 @@ public class BlockTermsReader extends FieldsProducer {
//private String segment;
public BlockTermsReader(TermsIndexReaderBase indexReader, Directory dir, FieldInfos fieldInfos, String segment, PostingsReaderBase postingsReader, int readBufferSize,
public BlockTermsReader(TermsIndexReaderBase indexReader, Directory dir, FieldInfos fieldInfos, String segment, PostingsReaderBase postingsReader, IOContext context,
int termsCacheSize, int codecId)
throws IOException {
@ -116,7 +117,7 @@ public class BlockTermsReader extends FieldsProducer {
//this.segment = segment;
in = dir.openInput(IndexFileNames.segmentFileName(segment, codecId, BlockTermsWriter.TERMS_EXTENSION),
readBufferSize);
context);
boolean success = false;
try {

View File

@ -72,7 +72,7 @@ public class BlockTermsWriter extends FieldsConsumer {
throws IOException {
final String termsFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, TERMS_EXTENSION);
this.termsIndexWriter = termsIndexWriter;
out = state.directory.createOutput(termsFileName);
out = state.directory.createOutput(termsFileName, state.context);
boolean success = false;
try {
fieldInfos = state.fieldInfos;

View File

@ -29,6 +29,7 @@ import org.apache.lucene.index.PerDocWriteState;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.values.Writer;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.BytesRef;
/**
@ -42,13 +43,17 @@ public class DefaultDocValuesConsumer extends PerDocConsumer {
private final AtomicLong bytesUsed;
private final Comparator<BytesRef> comparator;
private boolean useCompoundFile;
private final IOContext context;
public DefaultDocValuesConsumer(PerDocWriteState state, Comparator<BytesRef> comparator, boolean useCompoundFile) throws IOException {
this.segmentName = state.segmentName;
this.codecId = state.codecId;
this.bytesUsed = state.bytesUsed;
this.context = state.context;
//TODO maybe we should enable a global CFS that all codecs can pull on demand to further reduce the number of files?
this.directory = useCompoundFile ? state.directory.createCompoundOutput(IndexFileNames.segmentFileName(segmentName, state.codecId, IndexFileNames.COMPOUND_FILE_EXTENSION)) : state.directory;
this.directory = useCompoundFile ? state.directory.createCompoundOutput(
IndexFileNames.segmentFileName(segmentName, codecId,
IndexFileNames.COMPOUND_FILE_EXTENSION), context) : state.directory;
this.comparator = comparator;
this.useCompoundFile = useCompoundFile;
}
@ -63,7 +68,7 @@ public class DefaultDocValuesConsumer extends PerDocConsumer {
public DocValuesConsumer addValuesField(FieldInfo field) throws IOException {
return Writer.create(field.getDocValues(),
docValuesId(segmentName, codecId, field.number),
directory, comparator, bytesUsed);
directory, comparator, bytesUsed, context);
}
@SuppressWarnings("fallthrough")

View File

@ -33,6 +33,7 @@ import org.apache.lucene.index.values.Floats;
import org.apache.lucene.index.values.Ints;
import org.apache.lucene.index.values.ValueType;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
@ -72,17 +73,17 @@ public class DefaultDocValuesProducer extends PerDocValues {
* if an {@link IOException} occurs
*/
public DefaultDocValuesProducer(SegmentInfo si, Directory dir,
FieldInfos fieldInfo, int codecId, boolean useCompoundFile, Comparator<BytesRef> sortComparator) throws IOException {
FieldInfos fieldInfo, int codecId, boolean useCompoundFile, Comparator<BytesRef> sortComparator, IOContext context) throws IOException {
this.useCompoundFile = useCompoundFile;
this.sortComparator = sortComparator;
final Directory directory;
if (useCompoundFile) {
cfs = directory = dir.openCompoundInput(IndexFileNames.segmentFileName(si.name, codecId, IndexFileNames.COMPOUND_FILE_EXTENSION), 1024);
cfs = directory = dir.openCompoundInput(IndexFileNames.segmentFileName(si.name, codecId, IndexFileNames.COMPOUND_FILE_EXTENSION), context);
} else {
cfs = null;
directory = dir;
}
docValues = load(fieldInfo, si.name, si.docCount, directory, codecId);
docValues = load(fieldInfo, si.name, si.docCount, directory, codecId, context);
}
/**
@ -96,7 +97,7 @@ public class DefaultDocValuesProducer extends PerDocValues {
// Only opens files... doesn't actually load any values
protected TreeMap<String, IndexDocValues> load(FieldInfos fieldInfos,
String segment, int docCount, Directory dir, int codecId)
String segment, int docCount, Directory dir, int codecId, IOContext context)
throws IOException {
TreeMap<String, IndexDocValues> values = new TreeMap<String, IndexDocValues>();
boolean success = false;
@ -110,7 +111,7 @@ public class DefaultDocValuesProducer extends PerDocValues {
final String id = DefaultDocValuesConsumer.docValuesId(segment,
codecId, fieldInfo.number);
values.put(field,
loadDocValues(docCount, dir, id, fieldInfo.getDocValues(), sortComparator));
loadDocValues(docCount, dir, id, fieldInfo.getDocValues(), sortComparator, context));
}
}
success = true;
@ -145,30 +146,30 @@ public class DefaultDocValuesProducer extends PerDocValues {
* if the given {@link ValueType} is not supported
*/
protected IndexDocValues loadDocValues(int docCount, Directory dir, String id,
ValueType type, Comparator<BytesRef> sortComparator) throws IOException {
ValueType type, Comparator<BytesRef> sortComparator, IOContext context) throws IOException {
switch (type) {
case FIXED_INTS_16:
case FIXED_INTS_32:
case FIXED_INTS_64:
case FIXED_INTS_8:
case VAR_INTS:
return Ints.getValues(dir, id, docCount);
return Ints.getValues(dir, id, docCount, context);
case FLOAT_32:
return Floats.getValues(dir, id, docCount);
return Floats.getValues(dir, id, docCount, context);
case FLOAT_64:
return Floats.getValues(dir, id, docCount);
return Floats.getValues(dir, id, docCount, context);
case BYTES_FIXED_STRAIGHT:
return Bytes.getValues(dir, id, Bytes.Mode.STRAIGHT, true, docCount, sortComparator);
return Bytes.getValues(dir, id, Bytes.Mode.STRAIGHT, true, docCount, sortComparator, context);
case BYTES_FIXED_DEREF:
return Bytes.getValues(dir, id, Bytes.Mode.DEREF, true, docCount, sortComparator);
return Bytes.getValues(dir, id, Bytes.Mode.DEREF, true, docCount, sortComparator, context);
case BYTES_FIXED_SORTED:
return Bytes.getValues(dir, id, Bytes.Mode.SORTED, true, docCount, sortComparator);
return Bytes.getValues(dir, id, Bytes.Mode.SORTED, true, docCount, sortComparator, context);
case BYTES_VAR_STRAIGHT:
return Bytes.getValues(dir, id, Bytes.Mode.STRAIGHT, false, docCount, sortComparator);
return Bytes.getValues(dir, id, Bytes.Mode.STRAIGHT, false, docCount, sortComparator, context);
case BYTES_VAR_DEREF:
return Bytes.getValues(dir, id, Bytes.Mode.DEREF, false, docCount, sortComparator);
return Bytes.getValues(dir, id, Bytes.Mode.DEREF, false, docCount, sortComparator, context);
case BYTES_VAR_SORTED:
return Bytes.getValues(dir, id, Bytes.Mode.SORTED, false, docCount, sortComparator);
return Bytes.getValues(dir, id, Bytes.Mode.SORTED, false, docCount, sortComparator, context);
default:
throw new IllegalStateException("unrecognized index values mode " + type);
}

View File

@ -28,6 +28,7 @@ import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentInfos;
import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
/**
@ -38,10 +39,10 @@ public class DefaultSegmentInfosReader extends SegmentInfosReader {
@Override
public void read(Directory directory, String segmentsFileName, CodecProvider codecs,
SegmentInfos infos) throws IOException {
SegmentInfos infos, IOContext context) throws IOException {
IndexInput input = null;
try {
input = openInput(directory, segmentsFileName);
input = openInput(directory, segmentsFileName, context);
final int format = input.readInt();
infos.setFormat(format);
@ -69,11 +70,11 @@ public class DefaultSegmentInfosReader extends SegmentInfosReader {
if (si.getDocStoreIsCompoundFile()) {
dir = dir.openCompoundInput(IndexFileNames.segmentFileName(
si.getDocStoreSegment(), "",
IndexFileNames.COMPOUND_FILE_STORE_EXTENSION), 1024);
IndexFileNames.COMPOUND_FILE_STORE_EXTENSION), context);
}
} else if (si.getUseCompoundFile()) {
dir = dir.openCompoundInput(IndexFileNames.segmentFileName(
si.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION), 1024);
si.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION), context);
}
try {
@ -107,8 +108,8 @@ public class DefaultSegmentInfosReader extends SegmentInfosReader {
}
public IndexInput openInput(Directory dir, String segmentsFileName) throws IOException {
IndexInput in = dir.openInput(segmentsFileName);
public IndexInput openInput(Directory dir, String segmentsFileName, IOContext context) throws IOException {
IndexInput in = dir.openInput(segmentsFileName, context);
return new ChecksumIndexInput(in);
}

View File

@ -23,6 +23,8 @@ import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentInfos;
import org.apache.lucene.store.ChecksumIndexOutput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FlushInfo;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.IOUtils;
@ -54,9 +56,9 @@ public class DefaultSegmentInfosWriter extends SegmentInfosWriter {
public static final int FORMAT_MINIMUM = FORMAT_DIAGNOSTICS;
@Override
public IndexOutput writeInfos(Directory dir, String segmentFileName, SegmentInfos infos)
public IndexOutput writeInfos(Directory dir, String segmentFileName, SegmentInfos infos, IOContext context)
throws IOException {
IndexOutput out = createOutput(dir, segmentFileName);
IndexOutput out = createOutput(dir, segmentFileName, new IOContext(new FlushInfo(infos.size(), infos.totalDocCount())));
boolean success = false;
try {
out.writeInt(FORMAT_CURRENT); // write FORMAT
@ -77,9 +79,9 @@ public class DefaultSegmentInfosWriter extends SegmentInfosWriter {
}
}
protected IndexOutput createOutput(Directory dir, String segmentFileName)
protected IndexOutput createOutput(Directory dir, String segmentFileName, IOContext context)
throws IOException {
IndexOutput plainOut = dir.createOutput(segmentFileName);
IndexOutput plainOut = dir.createOutput(segmentFileName, context);
ChecksumIndexOutput out = new ChecksumIndexOutput(plainOut);
return out;
}

View File

@ -18,7 +18,9 @@ package org.apache.lucene.index.codecs;
*/
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IOContext.Context;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.SegmentInfo;
@ -68,12 +70,12 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
// start of the field info data
protected long dirOffset;
public FixedGapTermsIndexReader(Directory dir, FieldInfos fieldInfos, String segment, int indexDivisor, Comparator<BytesRef> termComp, int codecId)
public FixedGapTermsIndexReader(Directory dir, FieldInfos fieldInfos, String segment, int indexDivisor, Comparator<BytesRef> termComp, int codecId, IOContext context)
throws IOException {
this.termComp = termComp;
in = dir.openInput(IndexFileNames.segmentFileName(segment, codecId, FixedGapTermsIndexWriter.TERMS_INDEX_EXTENSION));
in = dir.openInput(IndexFileNames.segmentFileName(segment, codecId, FixedGapTermsIndexWriter.TERMS_INDEX_EXTENSION), context);
boolean success = false;

View File

@ -58,7 +58,7 @@ public class FixedGapTermsIndexWriter extends TermsIndexWriterBase {
public FixedGapTermsIndexWriter(SegmentWriteState state) throws IOException {
final String indexFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, TERMS_INDEX_EXTENSION);
termIndexInterval = state.termIndexInterval;
out = state.directory.createOutput(indexFileName);
out = state.directory.createOutput(indexFileName, state.context);
boolean success = false;
try {
fieldInfos = state.fieldInfos;

View File

@ -21,6 +21,7 @@ import java.io.IOException;
import org.apache.lucene.index.SegmentInfos;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
/**
* Specifies an API for classes that can read {@link SegmentInfos} information.
@ -36,5 +37,5 @@ public abstract class SegmentInfosReader {
* @param infos empty instance to be populated with data
* @throws IOException
*/
public abstract void read(Directory directory, String segmentsFileName, CodecProvider codecs, SegmentInfos infos) throws IOException;
public abstract void read(Directory directory, String segmentsFileName, CodecProvider codecs, SegmentInfos infos, IOContext context) throws IOException;
}

View File

@ -21,6 +21,7 @@ import java.io.IOException;
import org.apache.lucene.index.SegmentInfos;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
/**
@ -41,7 +42,7 @@ public abstract class SegmentInfosWriter {
* phase commit" operations as described above.
* @throws IOException
*/
public abstract IndexOutput writeInfos(Directory dir, String segmentsFileName, SegmentInfos infos) throws IOException;
public abstract IndexOutput writeInfos(Directory dir, String segmentsFileName, SegmentInfos infos, IOContext context) throws IOException;
/**
* First phase of the two-phase commit - ensure that all output can be

View File

@ -30,7 +30,9 @@ import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IOContext.Context;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CodecUtil;
import org.apache.lucene.util.fst.Builder;
@ -57,10 +59,10 @@ public class VariableGapTermsIndexReader extends TermsIndexReaderBase {
protected long dirOffset;
final String segment;
public VariableGapTermsIndexReader(Directory dir, FieldInfos fieldInfos, String segment, int indexDivisor, int codecId)
public VariableGapTermsIndexReader(Directory dir, FieldInfos fieldInfos, String segment, int indexDivisor, int codecId, IOContext context)
throws IOException {
in = dir.openInput(IndexFileNames.segmentFileName(segment, codecId, VariableGapTermsIndexWriter.TERMS_INDEX_EXTENSION));
in = dir.openInput(IndexFileNames.segmentFileName(segment, codecId, VariableGapTermsIndexWriter.TERMS_INDEX_EXTENSION), context);
this.segment = segment;
boolean success = false;

View File

@ -159,7 +159,7 @@ public class VariableGapTermsIndexWriter extends TermsIndexWriterBase {
public VariableGapTermsIndexWriter(SegmentWriteState state, IndexTermSelector policy) throws IOException {
final String indexFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, TERMS_INDEX_EXTENSION);
out = state.directory.createOutput(indexFileName);
out = state.directory.createOutput(indexFileName, state.context);
boolean success = false;
try {
fieldInfos = state.fieldInfos;

View File

@ -48,6 +48,7 @@ import org.apache.lucene.index.codecs.TermStats;
import org.apache.lucene.index.codecs.TermsConsumer;
import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.RAMOutputStream;
@ -242,7 +243,7 @@ public class MemoryCodec extends Codec {
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
final String fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, EXTENSION);
final IndexOutput out = state.directory.createOutput(fileName);
final IndexOutput out = state.directory.createOutput(fileName, state.context);
return new FieldsConsumer() {
@Override
@ -717,7 +718,7 @@ public class MemoryCodec extends Codec {
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
final String fileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.codecId, EXTENSION);
final IndexInput in = state.dir.openInput(fileName);
final IndexInput in = state.dir.openInput(fileName, IOContext.READONCE);
final SortedMap<String,TermsReader> fields = new TreeMap<String,TermsReader>();
@ -794,6 +795,6 @@ public class MemoryCodec extends Codec {
@Override
public PerDocValues docsProducer(SegmentReadState state) throws IOException {
return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId, getDocValuesUseCFS(), getDocValuesSortComparator());
return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId, getDocValuesUseCFS(), getDocValuesSortComparator(), IOContext.READONCE);
}
}

View File

@ -65,7 +65,7 @@ public class PreFlexCodec extends Codec {
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
return new PreFlexFields(state.dir, state.fieldInfos, state.segmentInfo, state.readBufferSize, state.termsIndexDivisor);
return new PreFlexFields(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.termsIndexDivisor);
}
@Override

View File

@ -38,6 +38,7 @@ import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.index.codecs.FieldsProducer;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
@ -62,10 +63,10 @@ public class PreFlexFields extends FieldsProducer {
final TreeMap<String,FieldInfo> fields = new TreeMap<String,FieldInfo>();
final Map<String,Terms> preTerms = new HashMap<String,Terms>();
private final Directory dir;
private final int readBufferSize;
private final IOContext context;
private Directory cfsReader;
public PreFlexFields(Directory dir, FieldInfos fieldInfos, SegmentInfo info, int readBufferSize, int indexDivisor)
public PreFlexFields(Directory dir, FieldInfos fieldInfos, SegmentInfo info, IOContext context, int indexDivisor)
throws IOException {
si = info;
@ -80,19 +81,19 @@ public class PreFlexFields extends FieldsProducer {
boolean success = false;
try {
TermInfosReader r = new TermInfosReader(dir, info.name, fieldInfos, readBufferSize, indexDivisor);
TermInfosReader r = new TermInfosReader(dir, info.name, fieldInfos, context, indexDivisor);
if (indexDivisor == -1) {
tisNoIndex = r;
} else {
tisNoIndex = null;
tis = r;
}
this.readBufferSize = readBufferSize;
this.context = context;
this.fieldInfos = fieldInfos;
// make sure that all index files have been read or are kept open
// so that if an index update removes them we'll still have them
freqStream = dir.openInput(IndexFileNames.segmentFileName(info.name, "", PreFlexCodec.FREQ_EXTENSION), readBufferSize);
freqStream = dir.openInput(IndexFileNames.segmentFileName(info.name, "", PreFlexCodec.FREQ_EXTENSION), context);
boolean anyProx = false;
for (FieldInfo fi : fieldInfos) {
if (fi.isIndexed) {
@ -105,7 +106,7 @@ public class PreFlexFields extends FieldsProducer {
}
if (anyProx) {
proxStream = dir.openInput(IndexFileNames.segmentFileName(info.name, "", PreFlexCodec.PROX_EXTENSION), readBufferSize);
proxStream = dir.openInput(IndexFileNames.segmentFileName(info.name, "", PreFlexCodec.PROX_EXTENSION), context);
} else {
proxStream = null;
}
@ -178,7 +179,7 @@ public class PreFlexFields extends FieldsProducer {
// to CFS
if (!(dir instanceof CompoundFileDirectory)) {
dir0 = cfsReader = dir.openCompoundInput(IndexFileNames.segmentFileName(si.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION), readBufferSize);
dir0 = cfsReader = dir.openCompoundInput(IndexFileNames.segmentFileName(si.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION), context);
} else {
dir0 = dir;
}
@ -187,7 +188,7 @@ public class PreFlexFields extends FieldsProducer {
dir0 = dir;
}
tis = new TermInfosReader(dir0, si.name, fieldInfos, readBufferSize, indexDivisor);
tis = new TermInfosReader(dir0, si.name, fieldInfos, context, indexDivisor);
}
}

View File

@ -25,6 +25,7 @@ import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.Term;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CloseableThreadLocal;
import org.apache.lucene.util.DoubleBarrelLRUCache;
@ -96,7 +97,7 @@ public final class TermInfosReader {
SegmentTermEnum termEnum;
}
TermInfosReader(Directory dir, String seg, FieldInfos fis, int readBufferSize, int indexDivisor)
TermInfosReader(Directory dir, String seg, FieldInfos fis, IOContext context, int indexDivisor)
throws CorruptIndexException, IOException {
boolean success = false;
@ -110,7 +111,7 @@ public final class TermInfosReader {
fieldInfos = fis;
origEnum = new SegmentTermEnum(directory.openInput(IndexFileNames.segmentFileName(segment, "", PreFlexCodec.TERMS_EXTENSION),
readBufferSize), fieldInfos, false);
context), fieldInfos, false);
size = origEnum.size;
@ -118,7 +119,7 @@ public final class TermInfosReader {
// Load terms index
totalIndexInterval = origEnum.indexInterval * indexDivisor;
final SegmentTermEnum indexEnum = new SegmentTermEnum(directory.openInput(IndexFileNames.segmentFileName(segment, "", PreFlexCodec.TERMS_INDEX_EXTENSION),
readBufferSize), fieldInfos, true);
context), fieldInfos, true);
try {
int indexSize = 1+((int)indexEnum.size-1)/indexDivisor; // otherwise read index

View File

@ -118,7 +118,7 @@ public class PulsingCodec extends Codec {
// We wrap StandardPostingsReader, but any StandardPostingsReader
// will work:
PostingsReaderBase docsReader = new StandardPostingsReader(state.dir, state.segmentInfo, state.readBufferSize, state.codecId);
PostingsReaderBase docsReader = new StandardPostingsReader(state.dir, state.segmentInfo, state.context, state.codecId);
PostingsReaderBase pulsingReader = new PulsingPostingsReaderImpl(docsReader);
// Terms dict index reader
@ -130,7 +130,7 @@ public class PulsingCodec extends Codec {
state.fieldInfos,
state.segmentInfo.name,
state.termsIndexDivisor,
state.codecId);
state.codecId, state.context);
success = true;
} finally {
if (!success) {
@ -144,7 +144,7 @@ public class PulsingCodec extends Codec {
FieldsProducer ret = new BlockTermsReader(indexReader,
state.dir, state.fieldInfos, state.segmentInfo.name,
pulsingReader,
state.readBufferSize,
state.context,
StandardCodec.TERMS_CACHE_SIZE,
state.codecId);
success = true;
@ -181,6 +181,6 @@ public class PulsingCodec extends Codec {
@Override
public PerDocValues docsProducer(SegmentReadState state) throws IOException {
return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId, getDocValuesUseCFS(), getDocValuesSortComparator());
return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId, getDocValuesUseCFS(), getDocValuesSortComparator(), state.context);
}
}

View File

@ -18,16 +18,16 @@ package org.apache.lucene.index.codecs.sep;
*/
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.BufferedIndexInput;
import org.apache.lucene.store.IOContext;
import java.io.IOException;
/** @lucene.experimental */
public abstract class IntStreamFactory {
public IntIndexInput openInput(Directory dir, String fileName) throws IOException {
return openInput(dir, fileName, BufferedIndexInput.BUFFER_SIZE);
public IntIndexInput openInput(Directory dir, String fileName, IOContext context) throws IOException {
return openInput(dir, fileName, context);
}
public abstract IntIndexInput openInput(Directory dir, String fileName, int readBufferSize) throws IOException;
public abstract IntIndexOutput createOutput(Directory dir, String fileName) throws IOException;
// public abstract IntIndexInput openInput(Directory dir, String fileName, IOContext context) throws IOException;
public abstract IntIndexOutput createOutput(Directory dir, String fileName, IOContext context) throws IOException;
}

View File

@ -30,6 +30,7 @@ import org.apache.lucene.index.codecs.BlockTermState;
import org.apache.lucene.index.codecs.PostingsReaderBase;
import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Bits;
@ -58,19 +59,19 @@ public class SepPostingsReaderImpl extends PostingsReaderBase {
int maxSkipLevels;
int skipMinimum;
public SepPostingsReaderImpl(Directory dir, SegmentInfo segmentInfo, int readBufferSize, IntStreamFactory intFactory, int codecId) throws IOException {
public SepPostingsReaderImpl(Directory dir, SegmentInfo segmentInfo, IOContext context, IntStreamFactory intFactory, int codecId) throws IOException {
boolean success = false;
try {
final String docFileName = IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriterImpl.DOC_EXTENSION);
docIn = intFactory.openInput(dir, docFileName);
docIn = intFactory.openInput(dir, docFileName, context);
skipIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriterImpl.SKIP_EXTENSION), readBufferSize);
skipIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriterImpl.SKIP_EXTENSION), context);
if (segmentInfo.getHasProx()) {
freqIn = intFactory.openInput(dir, IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriterImpl.FREQ_EXTENSION));
posIn = intFactory.openInput(dir, IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriterImpl.POS_EXTENSION), readBufferSize);
payloadIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriterImpl.PAYLOAD_EXTENSION), readBufferSize);
freqIn = intFactory.openInput(dir, IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriterImpl.FREQ_EXTENSION), context);
posIn = intFactory.openInput(dir, IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriterImpl.POS_EXTENSION), context);
payloadIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriterImpl.PAYLOAD_EXTENSION), context);
} else {
posIn = null;
payloadIn = null;

View File

@ -118,25 +118,25 @@ public final class SepPostingsWriterImpl extends PostingsWriterBase {
this.skipInterval = skipInterval;
this.skipMinimum = skipInterval; /* set to the same for now */
final String docFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, DOC_EXTENSION);
docOut = factory.createOutput(state.directory, docFileName);
docOut = factory.createOutput(state.directory, docFileName, state.context);
docIndex = docOut.index();
if (state.fieldInfos.hasProx()) {
final String frqFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, FREQ_EXTENSION);
freqOut = factory.createOutput(state.directory, frqFileName);
freqOut = factory.createOutput(state.directory, frqFileName, state.context);
freqIndex = freqOut.index();
final String posFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, POS_EXTENSION);
posOut = factory.createOutput(state.directory, posFileName);
posOut = factory.createOutput(state.directory, posFileName, state.context);
posIndex = posOut.index();
// TODO: -- only if at least one field stores payloads?
final String payloadFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, PAYLOAD_EXTENSION);
payloadOut = state.directory.createOutput(payloadFileName);
payloadOut = state.directory.createOutput(payloadFileName, state.context);
}
final String skipFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, SKIP_EXTENSION);
skipOut = state.directory.createOutput(skipFileName);
skipOut = state.directory.createOutput(skipFileName, state.context);
totalNumDocs = state.numDocs;

View File

@ -86,6 +86,6 @@ public class SimpleTextCodec extends Codec {
@Override
public PerDocValues docsProducer(SegmentReadState state) throws IOException {
return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId, getDocValuesUseCFS(), getDocValuesSortComparator());
return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId, getDocValuesUseCFS(), getDocValuesSortComparator(), state.context);
}
}

View File

@ -57,7 +57,7 @@ class SimpleTextFieldsReader extends FieldsProducer {
final static BytesRef PAYLOAD = SimpleTextFieldsWriter.PAYLOAD;
public SimpleTextFieldsReader(SegmentReadState state) throws IOException {
in = state.dir.openInput(SimpleTextCodec.getPostingsFileName(state.segmentInfo.name, state.codecId));
in = state.dir.openInput(SimpleTextCodec.getPostingsFileName(state.segmentInfo.name, state.codecId), state.context);
fieldInfos = state.fieldInfos;
}

View File

@ -46,7 +46,7 @@ class SimpleTextFieldsWriter extends FieldsConsumer {
public SimpleTextFieldsWriter(SegmentWriteState state) throws IOException {
final String fileName = SimpleTextCodec.getPostingsFileName(state.segmentName, state.codecId);
out = state.directory.createOutput(fileName);
out = state.directory.createOutput(fileName, state.context);
}
private void write(String s) throws IOException {

View File

@ -88,7 +88,7 @@ public class StandardCodec extends Codec {
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postings = new StandardPostingsReader(state.dir, state.segmentInfo, state.readBufferSize, state.codecId);
PostingsReaderBase postings = new StandardPostingsReader(state.dir, state.segmentInfo, state.context, state.codecId);
TermsIndexReaderBase indexReader;
boolean success = false;
@ -97,7 +97,7 @@ public class StandardCodec extends Codec {
state.fieldInfos,
state.segmentInfo.name,
state.termsIndexDivisor,
state.codecId);
state.codecId, state.context);
success = true;
} finally {
if (!success) {
@ -112,7 +112,7 @@ public class StandardCodec extends Codec {
state.fieldInfos,
state.segmentInfo.name,
postings,
state.readBufferSize,
state.context,
TERMS_CACHE_SIZE,
state.codecId);
success = true;
@ -162,6 +162,6 @@ public class StandardCodec extends Codec {
@Override
public PerDocValues docsProducer(SegmentReadState state) throws IOException {
return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId, getDocValuesUseCFS(), getDocValuesSortComparator());
return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId, getDocValuesUseCFS(), getDocValuesSortComparator(), state.context);
}
}

View File

@ -30,6 +30,7 @@ import org.apache.lucene.index.codecs.BlockTermState;
import org.apache.lucene.index.codecs.PostingsReaderBase;
import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Bits;
@ -51,15 +52,15 @@ public class StandardPostingsReader extends PostingsReaderBase {
//private String segment;
public StandardPostingsReader(Directory dir, SegmentInfo segmentInfo, int readBufferSize, int codecId) throws IOException {
public StandardPostingsReader(Directory dir, SegmentInfo segmentInfo, IOContext context, int codecId) throws IOException {
freqIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, StandardCodec.FREQ_EXTENSION),
readBufferSize);
context);
//this.segment = segmentInfo.name;
if (segmentInfo.getHasProx()) {
boolean success = false;
try {
proxIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, StandardCodec.PROX_EXTENSION),
readBufferSize);
context);
success = true;
} finally {
if (!success) {

View File

@ -92,14 +92,14 @@ public final class StandardPostingsWriter extends PostingsWriterBase {
this.skipMinimum = skipInterval; /* set to the same for now */
//this.segment = state.segmentName;
String fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, StandardCodec.FREQ_EXTENSION);
freqOut = state.directory.createOutput(fileName);
freqOut = state.directory.createOutput(fileName, state.context);
boolean success = false;
try {
if (state.fieldInfos.hasProx()) {
// At least one field does not omit TF, so create the
// prox file
fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, StandardCodec.PROX_EXTENSION);
proxOut = state.directory.createOutput(fileName);
proxOut = state.directory.createOutput(fileName, state.context);
} else {
// Every field omits TF so we will write no prox file
proxOut = null;

View File

@ -28,6 +28,7 @@ import org.apache.lucene.index.values.IndexDocValues.SortedSource;
import org.apache.lucene.index.values.IndexDocValues.Source;
import org.apache.lucene.index.values.IndexDocValues.SourceEnum;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.AttributeSource;
@ -100,12 +101,13 @@ public final class Bytes {
* {@link Writer}. A call to {@link Writer#finish(int)} will release
* all internally used resources and frees the memeory tracking
* reference.
* @param context
* @return a new {@link Writer} instance
* @throws IOException
* if the files for the writer can not be created.
*/
public static Writer getWriter(Directory dir, String id, Mode mode,
Comparator<BytesRef> comp, boolean fixedSize, AtomicLong bytesUsed)
Comparator<BytesRef> comp, boolean fixedSize, AtomicLong bytesUsed, IOContext context)
throws IOException {
// TODO -- i shouldn't have to specify fixed? can
// track itself & do the write thing at write time?
@ -115,19 +117,19 @@ public final class Bytes {
if (fixedSize) {
if (mode == Mode.STRAIGHT) {
return new FixedStraightBytesImpl.Writer(dir, id, bytesUsed);
return new FixedStraightBytesImpl.Writer(dir, id, bytesUsed, context);
} else if (mode == Mode.DEREF) {
return new FixedDerefBytesImpl.Writer(dir, id, bytesUsed);
return new FixedDerefBytesImpl.Writer(dir, id, bytesUsed, context);
} else if (mode == Mode.SORTED) {
return new FixedSortedBytesImpl.Writer(dir, id, comp, bytesUsed);
return new FixedSortedBytesImpl.Writer(dir, id, comp, bytesUsed, context);
}
} else {
if (mode == Mode.STRAIGHT) {
return new VarStraightBytesImpl.Writer(dir, id, bytesUsed);
return new VarStraightBytesImpl.Writer(dir, id, bytesUsed, context);
} else if (mode == Mode.DEREF) {
return new VarDerefBytesImpl.Writer(dir, id, bytesUsed);
return new VarDerefBytesImpl.Writer(dir, id, bytesUsed, context);
} else if (mode == Mode.SORTED) {
return new VarSortedBytesImpl.Writer(dir, id, comp, bytesUsed);
return new VarSortedBytesImpl.Writer(dir, id, comp, bytesUsed, context);
}
}
@ -157,23 +159,24 @@ public final class Bytes {
* if an {@link IOException} occurs
*/
public static IndexDocValues getValues(Directory dir, String id, Mode mode,
boolean fixedSize, int maxDoc, Comparator<BytesRef> sortComparator) throws IOException {
boolean fixedSize, int maxDoc, Comparator<BytesRef> sortComparator, IOContext context) throws IOException {
// TODO -- I can peek @ header to determing fixed/mode?
if (fixedSize) {
if (mode == Mode.STRAIGHT) {
return new FixedStraightBytesImpl.Reader(dir, id, maxDoc);
return new FixedStraightBytesImpl.Reader(dir, id, maxDoc, context);
} else if (mode == Mode.DEREF) {
return new FixedDerefBytesImpl.Reader(dir, id, maxDoc);
return new FixedDerefBytesImpl.Reader(dir, id, maxDoc, context);
} else if (mode == Mode.SORTED) {
return new FixedSortedBytesImpl.Reader(dir, id, maxDoc);
return new FixedSortedBytesImpl.Reader(dir, id, maxDoc, context);
}
} else {
if (mode == Mode.STRAIGHT) {
return new VarStraightBytesImpl.Reader(dir, id, maxDoc);
return new VarStraightBytesImpl.Reader(dir, id, maxDoc, context);
} else if (mode == Mode.DEREF) {
return new VarDerefBytesImpl.Reader(dir, id, maxDoc);
return new VarDerefBytesImpl.Reader(dir, id, maxDoc, context);
} else if (mode == Mode.SORTED) {
return new VarSortedBytesImpl.Reader(dir, id, maxDoc, sortComparator);
return new VarSortedBytesImpl.Reader(dir, id, maxDoc, sortComparator, context);
}
}
@ -343,15 +346,16 @@ public final class Bytes {
private final Directory dir;
private final String codecName;
private final int version;
private final IOContext context;
protected BytesWriterBase(Directory dir, String id, String codecName,
int version,
AtomicLong bytesUsed) throws IOException {
int version, AtomicLong bytesUsed, IOContext context) throws IOException {
super(bytesUsed);
this.id = id;
this.dir = dir;
this.codecName = codecName;
this.version = version;
this.context = context;
}
protected IndexOutput getDataOut() throws IOException {
@ -359,7 +363,7 @@ public final class Bytes {
boolean success = false;
try {
datOut = dir.createOutput(IndexFileNames.segmentFileName(id, "",
DATA_EXTENSION));
DATA_EXTENSION), context);
CodecUtil.writeHeader(datOut, codecName, version);
success = true;
} finally {
@ -376,7 +380,7 @@ public final class Bytes {
try {
if (idxOut == null) {
idxOut = dir.createOutput(IndexFileNames.segmentFileName(id, "",
INDEX_EXTENSION));
INDEX_EXTENSION), context);
CodecUtil.writeHeader(idxOut, codecName, version);
}
success = true;
@ -439,16 +443,16 @@ public final class Bytes {
protected final String id;
protected BytesReaderBase(Directory dir, String id, String codecName,
int maxVersion, boolean doIndex) throws IOException {
int maxVersion, boolean doIndex, IOContext context) throws IOException {
this.id = id;
datIn = dir.openInput(IndexFileNames.segmentFileName(id, "",
Writer.DATA_EXTENSION));
Writer.DATA_EXTENSION), context);
boolean success = false;
try {
version = CodecUtil.checkHeader(datIn, codecName, maxVersion, maxVersion);
if (doIndex) {
idxIn = dir.openInput(IndexFileNames.segmentFileName(id, "",
Writer.INDEX_EXTENSION));
Writer.INDEX_EXTENSION), context);
final int version2 = CodecUtil.checkHeader(idxIn, codecName,
maxVersion, maxVersion);
assert version == version2;

View File

@ -24,6 +24,7 @@ import org.apache.lucene.index.values.Bytes.BytesBaseSource;
import org.apache.lucene.index.values.Bytes.BytesReaderBase;
import org.apache.lucene.index.values.Bytes.BytesWriterBase;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.ArrayUtil;
@ -54,15 +55,15 @@ class FixedDerefBytesImpl {
private int size = -1;
private int[] docToID;
private final BytesRefHash hash;
public Writer(Directory dir, String id, AtomicLong bytesUsed)
public Writer(Directory dir, String id, AtomicLong bytesUsed, IOContext context)
throws IOException {
this(dir, id, new DirectTrackingAllocator(ByteBlockPool.BYTE_BLOCK_SIZE, bytesUsed),
bytesUsed);
bytesUsed, context);
}
public Writer(Directory dir, String id, Allocator allocator,
AtomicLong bytesUsed) throws IOException {
super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed);
AtomicLong bytesUsed, IOContext context) throws IOException {
super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context);
hash = new BytesRefHash(new ByteBlockPool(allocator),
BytesRefHash.DEFAULT_CAPACITY, new TrackingDirectBytesStartArray(
BytesRefHash.DEFAULT_CAPACITY, bytesUsed));
@ -144,8 +145,8 @@ class FixedDerefBytesImpl {
public static class Reader extends BytesReaderBase {
private final int size;
Reader(Directory dir, String id, int maxDoc) throws IOException {
super(dir, id, CODEC_NAME, VERSION_START, true);
Reader(Directory dir, String id, int maxDoc, IOContext context) throws IOException {
super(dir, id, CODEC_NAME, VERSION_START, true, context);
size = datIn.readInt();
}

View File

@ -26,6 +26,7 @@ import org.apache.lucene.index.values.Bytes.BytesReaderBase;
import org.apache.lucene.index.values.Bytes.BytesWriterBase;
import org.apache.lucene.index.values.FixedDerefBytesImpl.Reader.DerefBytesEnum;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.ArrayUtil;
@ -61,14 +62,14 @@ class FixedSortedBytesImpl {
private final BytesRefHash hash;
public Writer(Directory dir, String id, Comparator<BytesRef> comp,
AtomicLong bytesUsed) throws IOException {
AtomicLong bytesUsed, IOContext context) throws IOException {
this(dir, id, comp, new DirectTrackingAllocator(ByteBlockPool.BYTE_BLOCK_SIZE, bytesUsed),
bytesUsed);
bytesUsed, context);
}
public Writer(Directory dir, String id, Comparator<BytesRef> comp,
Allocator allocator, AtomicLong bytesUsed) throws IOException {
super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed);
Allocator allocator, AtomicLong bytesUsed, IOContext context) throws IOException {
super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context);
ByteBlockPool pool = new ByteBlockPool(allocator);
hash = new BytesRefHash(pool, BytesRefHash.DEFAULT_CAPACITY,
new TrackingDirectBytesStartArray(BytesRefHash.DEFAULT_CAPACITY,
@ -169,8 +170,8 @@ class FixedSortedBytesImpl {
public static class Reader extends BytesReaderBase {
private final int size;
public Reader(Directory dir, String id, int maxDoc) throws IOException {
super(dir, id, CODEC_NAME, VERSION_START, true);
public Reader(Directory dir, String id, int maxDoc, IOContext context) throws IOException {
super(dir, id, CODEC_NAME, VERSION_START, true, context);
size = datIn.readInt();
}

View File

@ -26,6 +26,7 @@ import org.apache.lucene.index.values.Bytes.BytesBaseSource;
import org.apache.lucene.index.values.Bytes.BytesReaderBase;
import org.apache.lucene.index.values.Bytes.BytesWriterBase;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.AttributeSource;
@ -55,8 +56,8 @@ class FixedStraightBytesImpl {
private final int byteBlockSize;
private IndexOutput datOut;
public Writer(Directory dir, String id, AtomicLong bytesUsed) throws IOException {
super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed);
public Writer(Directory dir, String id, AtomicLong bytesUsed, IOContext context) throws IOException {
super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context);
pool = new ByteBlockPool(new DirectTrackingAllocator(bytesUsed));
byteBlockSize = BYTE_BLOCK_SIZE;
}
@ -204,8 +205,8 @@ class FixedStraightBytesImpl {
private final int size;
private final int maxDoc;
Reader(Directory dir, String id, int maxDoc) throws IOException {
super(dir, id, CODEC_NAME, VERSION_START, false);
Reader(Directory dir, String id, int maxDoc, IOContext context) throws IOException {
super(dir, id, CODEC_NAME, VERSION_START, false, context);
size = datIn.readInt();
this.maxDoc = maxDoc;
}

View File

@ -23,6 +23,7 @@ import java.util.concurrent.atomic.AtomicLong;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.values.IndexDocValues.Source;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.ArrayUtil;
@ -49,21 +50,21 @@ public class Floats {
private static final byte[] DEFAULTS = new byte[] {0,0,0,0,0,0,0,0};
public static Writer getWriter(Directory dir, String id, int precisionBytes,
AtomicLong bytesUsed) throws IOException {
AtomicLong bytesUsed, IOContext context) throws IOException {
if (precisionBytes != 4 && precisionBytes != 8) {
throw new IllegalArgumentException("precisionBytes must be 4 or 8; got "
+ precisionBytes);
}
if (precisionBytes == 4) {
return new Float4Writer(dir, id, bytesUsed);
return new Float4Writer(dir, id, bytesUsed, context);
} else {
return new Float8Writer(dir, id, bytesUsed);
return new Float8Writer(dir, id, bytesUsed, context);
}
}
public static IndexDocValues getValues(Directory dir, String id, int maxDoc)
public static IndexDocValues getValues(Directory dir, String id, int maxDoc, IOContext context)
throws IOException {
return new FloatsReader(dir, id, maxDoc);
return new FloatsReader(dir, id, maxDoc, context);
}
abstract static class FloatsWriter extends Writer {
@ -73,13 +74,15 @@ public class Floats {
protected IndexOutput datOut;
private final byte precision;
private final Directory dir;
private final IOContext context;
protected FloatsWriter(Directory dir, String id, int precision,
AtomicLong bytesUsed) throws IOException {
AtomicLong bytesUsed, IOContext context) throws IOException {
super(bytesUsed);
this.id = id;
this.precision = (byte) precision;
this.dir = dir;
this.context = context;
}
@ -90,7 +93,7 @@ public class Floats {
final void initDataOut() throws IOException {
assert datOut == null;
datOut = dir.createOutput(IndexFileNames.segmentFileName(id, "",
Writer.DATA_EXTENSION));
Writer.DATA_EXTENSION), context);
boolean success = false;
try {
CodecUtil.writeHeader(datOut, CODEC_NAME, VERSION_CURRENT);
@ -158,9 +161,9 @@ public class Floats {
// Writes 4 bytes (float) per value
static final class Float4Writer extends FloatsWriter {
private int[] values;
protected Float4Writer(Directory dir, String id, AtomicLong bytesUsed)
protected Float4Writer(Directory dir, String id, AtomicLong bytesUsed, IOContext context)
throws IOException {
super(dir, id, 4, bytesUsed);
super(dir, id, 4, bytesUsed, context);
values = new int[1];
bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT);
}
@ -221,9 +224,9 @@ public class Floats {
// Writes 8 bytes (double) per value
static final class Float8Writer extends FloatsWriter {
private long[] values;
protected Float8Writer(Directory dir, String id, AtomicLong bytesUsed)
protected Float8Writer(Directory dir, String id, AtomicLong bytesUsed, IOContext context)
throws IOException {
super(dir, id, 8, bytesUsed);
super(dir, id, 8, bytesUsed, context);
values = new long[1];
bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_LONG);
}
@ -288,10 +291,10 @@ public class Floats {
// TODO(simonw) is ByteBuffer the way to go here?
private final int maxDoc;
protected FloatsReader(Directory dir, String id, int maxDoc)
protected FloatsReader(Directory dir, String id, int maxDoc, IOContext context)
throws IOException {
datIn = dir.openInput(IndexFileNames.segmentFileName(id, "",
Writer.DATA_EXTENSION));
Writer.DATA_EXTENSION), context);
CodecUtil.checkHeader(datIn, CODEC_NAME, VERSION_START, VERSION_START);
precisionBytes = datIn.readByte();
assert precisionBytes == 4 || precisionBytes == 8;

View File

@ -23,6 +23,7 @@ import java.util.concurrent.atomic.AtomicLong;
import org.apache.lucene.index.values.IntsImpl.IntsReader;
import org.apache.lucene.index.values.IntsImpl.IntsWriter;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
/**
* @lucene.experimental
@ -33,11 +34,13 @@ public class Ints {
private Ints() {
}
public static Writer getWriter(Directory dir, String id, AtomicLong bytesUsed, ValueType type) throws IOException {
return new IntsWriter(dir, id, bytesUsed, type);
public static Writer getWriter(Directory dir, String id,
AtomicLong bytesUsed, ValueType type, IOContext context) throws IOException {
return new IntsWriter(dir, id, bytesUsed, type, context);
}
public static IndexDocValues getValues(Directory dir, String id, int numDocs) throws IOException {
return new IntsReader(dir, id, numDocs);
public static IndexDocValues getValues(Directory dir, String id,
int numDocs, IOContext context) throws IOException {
return new IntsReader(dir, id, numDocs, context);
}
}

View File

@ -29,6 +29,7 @@ import org.apache.lucene.index.values.IndexDocValuesArray.IntValues;
import org.apache.lucene.index.values.IndexDocValuesArray.LongValues;
import org.apache.lucene.index.values.IndexDocValuesArray.ShortValues;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.AttributeSource;
@ -67,11 +68,13 @@ class IntsImpl {
private final byte typeOrd;
private IndexOutput datOut;
private boolean merging;
private final IOContext context;
protected IntsWriter(Directory dir, String id, AtomicLong bytesUsed,
ValueType valueType) throws IOException {
ValueType valueType, IOContext context) throws IOException {
super(bytesUsed);
this.context = context;
this.dir = dir;
this.id = id;
switch (valueType) {
@ -122,7 +125,7 @@ class IntsImpl {
boolean success = false;
try {
datOut = dir.createOutput(IndexFileNames.segmentFileName(id, "",
DATA_EXTENSION));
DATA_EXTENSION), context);
CodecUtil.writeHeader(datOut, CODEC_NAME, VERSION_CURRENT);
datOut.writeByte(typeOrd);
success = true;
@ -273,9 +276,9 @@ class IntsImpl {
private final byte type;
private final int numDocs;
protected IntsReader(Directory dir, String id, int numDocs) throws IOException {
protected IntsReader(Directory dir, String id, int numDocs, IOContext context) throws IOException {
datIn = dir.openInput(IndexFileNames.segmentFileName(id, "",
Writer.DATA_EXTENSION));
Writer.DATA_EXTENSION), context);
this.numDocs = numDocs;
boolean success = false;
try {

View File

@ -26,6 +26,7 @@ import org.apache.lucene.index.values.Bytes.BytesWriterBase;
import org.apache.lucene.index.values.FixedDerefBytesImpl.Reader.DerefBytesEnum;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.ArrayUtil;
@ -117,15 +118,15 @@ class VarDerefBytesImpl {
bytesUsed);
private final BytesRefHash hash;
public Writer(Directory dir, String id, AtomicLong bytesUsed)
public Writer(Directory dir, String id, AtomicLong bytesUsed, IOContext context)
throws IOException {
this(dir, id, new DirectTrackingAllocator(ByteBlockPool.BYTE_BLOCK_SIZE, bytesUsed),
bytesUsed);
bytesUsed, context);
}
public Writer(Directory dir, String id, Allocator allocator,
AtomicLong bytesUsed) throws IOException {
super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed);
AtomicLong bytesUsed, IOContext context) throws IOException {
super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context);
hash = new BytesRefHash(new ByteBlockPool(allocator), 16, array);
docToAddress = new int[1];
bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT);
@ -220,8 +221,8 @@ class VarDerefBytesImpl {
public static class Reader extends BytesReaderBase {
Reader(Directory dir, String id, int maxDoc) throws IOException {
super(dir, id, CODEC_NAME, VERSION_START, true);
Reader(Directory dir, String id, int maxDoc, IOContext context) throws IOException {
super(dir, id, CODEC_NAME, VERSION_START, true, context);
}
@Override

View File

@ -26,6 +26,7 @@ import org.apache.lucene.index.values.Bytes.BytesBaseSortedSource;
import org.apache.lucene.index.values.Bytes.BytesReaderBase;
import org.apache.lucene.index.values.Bytes.BytesWriterBase;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.ArrayUtil;
@ -61,14 +62,14 @@ class VarSortedBytesImpl {
private final BytesRefHash hash;
public Writer(Directory dir, String id, Comparator<BytesRef> comp,
AtomicLong bytesUsed) throws IOException {
AtomicLong bytesUsed, IOContext context) throws IOException {
this(dir, id, comp, new DirectTrackingAllocator(ByteBlockPool.BYTE_BLOCK_SIZE, bytesUsed),
bytesUsed);
bytesUsed, context);
}
public Writer(Directory dir, String id, Comparator<BytesRef> comp,
Allocator allocator, AtomicLong bytesUsed) throws IOException {
super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed);
Allocator allocator, AtomicLong bytesUsed, IOContext context) throws IOException {
super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context);
this.hash = new BytesRefHash(new ByteBlockPool(allocator),
BytesRefHash.DEFAULT_CAPACITY, new TrackingDirectBytesStartArray(
BytesRefHash.DEFAULT_CAPACITY, bytesUsed));
@ -168,8 +169,9 @@ class VarSortedBytesImpl {
public static class Reader extends BytesReaderBase {
private final Comparator<BytesRef> defaultComp;
Reader(Directory dir, String id, int maxDoc, Comparator<BytesRef> comparator) throws IOException {
super(dir, id, CODEC_NAME, VERSION_START, true);
Reader(Directory dir, String id, int maxDoc, Comparator<BytesRef> comparator, IOContext context) throws IOException {
super(dir, id, CODEC_NAME, VERSION_START, true, context);
this.defaultComp = comparator;
}

View File

@ -24,6 +24,7 @@ import org.apache.lucene.index.values.Bytes.BytesBaseSource;
import org.apache.lucene.index.values.Bytes.BytesReaderBase;
import org.apache.lucene.index.values.Bytes.BytesWriterBase;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.ArrayUtil;
@ -56,9 +57,9 @@ class VarStraightBytesImpl {
private final ByteBlockPool pool;
private IndexOutput datOut;
private boolean merge = false;
public Writer(Directory dir, String id, AtomicLong bytesUsed)
public Writer(Directory dir, String id, AtomicLong bytesUsed, IOContext context)
throws IOException {
super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed);
super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context);
pool = new ByteBlockPool(new DirectTrackingAllocator(bytesUsed));
docToAddress = new long[1];
pool.nextBuffer(); // init
@ -215,8 +216,8 @@ class VarStraightBytesImpl {
public static class Reader extends BytesReaderBase {
private final int maxDoc;
Reader(Directory dir, String id, int maxDoc) throws IOException {
super(dir, id, CODEC_NAME, VERSION_START, true);
Reader(Directory dir, String id, int maxDoc, IOContext context) throws IOException {
super(dir, id, CODEC_NAME, VERSION_START, true, context);
this.maxDoc = maxDoc;
}

View File

@ -22,6 +22,7 @@ import java.util.concurrent.atomic.AtomicLong;
import org.apache.lucene.index.codecs.DocValuesConsumer;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
@ -192,7 +193,7 @@ public abstract class Writer extends DocValuesConsumer {
* @throws IOException
*/
public static Writer create(ValueType type, String id, Directory directory,
Comparator<BytesRef> comp, AtomicLong bytesUsed) throws IOException {
Comparator<BytesRef> comp, AtomicLong bytesUsed, IOContext context) throws IOException {
if (comp == null) {
comp = BytesRef.getUTF8SortedAsUnicodeComparator();
}
@ -202,29 +203,29 @@ public abstract class Writer extends DocValuesConsumer {
case FIXED_INTS_64:
case FIXED_INTS_8:
case VAR_INTS:
return Ints.getWriter(directory, id, bytesUsed, type);
return Ints.getWriter(directory, id, bytesUsed, type, context);
case FLOAT_32:
return Floats.getWriter(directory, id, 4, bytesUsed);
return Floats.getWriter(directory, id, 4, bytesUsed, context);
case FLOAT_64:
return Floats.getWriter(directory, id, 8, bytesUsed);
return Floats.getWriter(directory, id, 8, bytesUsed, context);
case BYTES_FIXED_STRAIGHT:
return Bytes.getWriter(directory, id, Bytes.Mode.STRAIGHT, comp, true,
bytesUsed);
bytesUsed, context);
case BYTES_FIXED_DEREF:
return Bytes.getWriter(directory, id, Bytes.Mode.DEREF, comp, true,
bytesUsed);
bytesUsed, context);
case BYTES_FIXED_SORTED:
return Bytes.getWriter(directory, id, Bytes.Mode.SORTED, comp, true,
bytesUsed);
bytesUsed, context);
case BYTES_VAR_STRAIGHT:
return Bytes.getWriter(directory, id, Bytes.Mode.STRAIGHT, comp, false,
bytesUsed);
bytesUsed, context);
case BYTES_VAR_DEREF:
return Bytes.getWriter(directory, id, Bytes.Mode.DEREF, comp, false,
bytesUsed);
bytesUsed, context);
case BYTES_VAR_SORTED:
return Bytes.getWriter(directory, id, Bytes.Mode.SORTED, comp, false,
bytesUsed);
bytesUsed, context);
default:
throw new IllegalArgumentException("Unknown Values: " + type);

View File

@ -22,8 +22,19 @@ import java.io.IOException;
/** Base implementation class for buffered {@link IndexInput}. */
public abstract class BufferedIndexInput extends IndexInput {
/** Default buffer size */
/** Default buffer size set to 1024*/
public static final int BUFFER_SIZE = 1024;
// The normal read buffer size defaults to 1024, but
// increasing this during merging seems to yield
// performance gains. However we don't want to increase
// it too much because there are quite a few
// BufferedIndexInputs created during merging. See
// LUCENE-888 for details.
/**
* A buffer size for merges set to 4096
*/
public static final int MERGE_BUFFER_SIZE = 4096;
private int bufferSize = BUFFER_SIZE;
@ -41,6 +52,10 @@ public abstract class BufferedIndexInput extends IndexInput {
}
public BufferedIndexInput() {}
public BufferedIndexInput(IOContext context) {
this(bufferSize(context));
}
/** Inits BufferedIndexInput with a specific bufferSize */
public BufferedIndexInput(int bufferSize) {
@ -300,4 +315,21 @@ public abstract class BufferedIndexInput extends IndexInput {
}
}
/**
* Returns default buffer sizes for the given {@link IOContext}
*/
public static int bufferSize(IOContext context) {
switch (context.context) {
case DEFAULT:
case FLUSH:
case READ:
return BUFFER_SIZE;
case MERGE:
return MERGE_BUFFER_SIZE;
default:
assert false : "unknown IOContext " + context.context;
return BUFFER_SIZE;
}
}
}

View File

@ -48,7 +48,7 @@ public abstract class CompoundFileDirectory extends Directory {
private final Directory directory;
private final String fileName;
private final int readBufferSize;
protected final int readBufferSize;
private Map<String,FileEntry> entries;
private boolean openForWrite;
private static final Map<String,FileEntry> SENTINEL = Collections.emptyMap();
@ -59,11 +59,11 @@ public abstract class CompoundFileDirectory extends Directory {
* <p>
* NOTE: subclasses must call {@link #initForRead(Map)} before the directory can be used.
*/
public CompoundFileDirectory(Directory directory, String fileName, int readBufferSize) throws IOException {
public CompoundFileDirectory(Directory directory, String fileName, IOContext context) throws IOException {
this.directory = directory;
this.fileName = fileName;
this.readBufferSize = readBufferSize;
this.readBufferSize = BufferedIndexInput.bufferSize(context);
this.isOpen = false;
}
@ -91,7 +91,7 @@ public abstract class CompoundFileDirectory extends Directory {
IndexInput input = null;
try {
input = dir.openInput(IndexFileNames.segmentFileName(IndexFileNames.stripExtension(name), "",
IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION));
IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION), IOContext.READONCE);
final int readInt = input.readInt(); // unused right now
assert readInt == CompoundFileWriter.ENTRY_FORMAT_CURRENT;
final int numEntries = input.readVInt();
@ -189,13 +189,7 @@ public abstract class CompoundFileDirectory extends Directory {
}
@Override
public synchronized IndexInput openInput(String id) throws IOException {
// Default to readBufferSize passed in when we were opened
return openInput(id, readBufferSize);
}
@Override
public synchronized IndexInput openInput(String id, int readBufferSize) throws IOException {
public synchronized IndexInput openInput(String id, IOContext context) throws IOException {
ensureOpen();
assert !openForWrite;
id = IndexFileNames.stripSegmentName(id);
@ -273,9 +267,9 @@ public abstract class CompoundFileDirectory extends Directory {
}
@Override
public IndexOutput createOutput(String name) throws IOException {
public IndexOutput createOutput(String name, IOContext context) throws IOException {
ensureOpen();
return writer.createOutput(name);
return writer.createOutput(name, context);
}
@Override
@ -291,18 +285,18 @@ public abstract class CompoundFileDirectory extends Directory {
}
@Override
public CompoundFileDirectory openCompoundInput(String name, int bufferSize) throws IOException {
public CompoundFileDirectory openCompoundInput(String name, IOContext context) throws IOException {
FileEntry fileEntry = this.entries.get(IndexFileNames.stripSegmentName(name));
if (fileEntry == null) {
throw new FileNotFoundException("file " + name + " does not exists in this CFS");
}
return new NestedCompoundFileDirectory(name, bufferSize, fileEntry.offset, fileEntry.length);
return new NestedCompoundFileDirectory(name, context, fileEntry.offset, fileEntry.length);
}
/** Not implemented
* @throws UnsupportedOperationException */
@Override
public CompoundFileDirectory createCompoundOutput(String name)
public CompoundFileDirectory createCompoundOutput(String name, IOContext context)
throws IOException {
throw new UnsupportedOperationException("can not create nested CFS, create seperately and use Directory.copy instead");
}
@ -312,14 +306,14 @@ public abstract class CompoundFileDirectory extends Directory {
private final long cfsOffset;
private final long cfsLength;
public NestedCompoundFileDirectory(String fileName, int readBufferSize, long offset, long length)
public NestedCompoundFileDirectory(String fileName, IOContext context, long offset, long length)
throws IOException {
super(directory, fileName, readBufferSize);
super(directory, fileName, context);
this.cfsOffset = offset;
this.cfsLength = length;
IndexInput input = null;
try {
input = CompoundFileDirectory.this.openInput(fileName, 128);
input = CompoundFileDirectory.this.openInput(fileName, IOContext.READONCE);
initForRead(CompoundFileDirectory.readEntries(input,
CompoundFileDirectory.this, fileName));
} finally {

View File

@ -136,7 +136,7 @@ final class CompoundFileWriter implements Closeable{
IOException priorException = null;
IndexOutput entryTableOut = null;
try {
initDataOut();
initDataOut(IOContext.DEFAULT);
if (!pendingEntries.isEmpty() || outputTaken.get()) {
throw new IllegalStateException("CFS has pending open files");
}
@ -151,7 +151,7 @@ final class CompoundFileWriter implements Closeable{
IOUtils.closeSafely(priorException, dataOut);
}
try {
entryTableOut = directory.createOutput(entryTableName);
entryTableOut = directory.createOutput(entryTableName, IOContext.DEFAULT);
writeEntryTable(entries.values(), entryTableOut);
} catch (IOException e) {
priorException = e;
@ -180,7 +180,7 @@ final class CompoundFileWriter implements Closeable{
*/
private final long copyFileEntry(IndexOutput dataOut, FileEntry fileEntry)
throws IOException, MergeAbortedException {
final IndexInput is = fileEntry.dir.openInput(fileEntry.file);
final IndexInput is = fileEntry.dir.openInput(fileEntry.file, IOContext.READONCE);
try {
final long startPtr = dataOut.getFilePointer();
final long length = fileEntry.length;
@ -212,7 +212,7 @@ final class CompoundFileWriter implements Closeable{
}
}
IndexOutput createOutput(String name) throws IOException {
IndexOutput createOutput(String name, IOContext context) throws IOException {
ensureOpen();
boolean success = false;
try {
@ -225,7 +225,7 @@ final class CompoundFileWriter implements Closeable{
entries.put(name, entry);
final DirectCFSIndexOutput out;
if (outputTaken.compareAndSet(false, true)) {
initDataOut();
initDataOut(context);
success = true;
out = new DirectCFSIndexOutput(dataOut, entry, false);
} else {
@ -233,7 +233,7 @@ final class CompoundFileWriter implements Closeable{
if (directory.fileExists(name)) {
throw new IOException("File already exists");
}
out = new DirectCFSIndexOutput(directory.createOutput(name), entry,
out = new DirectCFSIndexOutput(directory.createOutput(name, context), entry,
true);
}
success = true;
@ -249,11 +249,11 @@ final class CompoundFileWriter implements Closeable{
outputTaken.compareAndSet(true, false);
}
private synchronized final void initDataOut() throws IOException {
private synchronized final void initDataOut(IOContext context) throws IOException {
if (dataOut == null) {
boolean success = false;
try {
dataOut = directory.createOutput(dataFileName);
dataOut = directory.createOutput(dataFileName, context);
dataOut.writeVInt(FORMAT_CURRENT);
success = true;
} finally {

View File

@ -31,11 +31,11 @@ import org.apache.lucene.util.IOUtils;
public class DefaultCompoundFileDirectory extends CompoundFileDirectory {
protected IndexInput stream;
public DefaultCompoundFileDirectory(Directory directory, String fileName, int readBufferSize, boolean writeable) throws IOException {
super(directory, fileName, readBufferSize);
public DefaultCompoundFileDirectory(Directory directory, String fileName, IOContext context, boolean writeable) throws IOException {
super(directory, fileName, context);
if (!writeable) {
try {
stream = directory.openInput(fileName, readBufferSize);
stream = directory.openInput(fileName, context);
initForRead(CompoundFileDirectory.readEntries(stream, directory, fileName));
} catch (IOException e) {
IOUtils.closeSafely(e, stream);

View File

@ -87,7 +87,7 @@ public abstract class Directory implements Closeable {
/** Creates a new, empty file in the directory with the given name.
Returns a stream writing this file. */
public abstract IndexOutput createOutput(String name)
public abstract IndexOutput createOutput(String name, IOContext context)
throws IOException;
/**
@ -103,10 +103,6 @@ public abstract class Directory implements Closeable {
*/
public abstract void sync(Collection<String> names) throws IOException;
/** Returns a stream reading an existing file. */
public abstract IndexInput openInput(String name)
throws IOException;
/** Returns a stream reading an existing file, with the
* specified read buffer size. The particular Directory
* implementation may ignore the buffer size. Currently
@ -114,9 +110,7 @@ public abstract class Directory implements Closeable {
* parameter are {@link FSDirectory} and {@link
* CompoundFileDirectory}.
*/
public IndexInput openInput(String name, int bufferSize) throws IOException {
return openInput(name);
}
public abstract IndexInput openInput(String name, IOContext context) throws IOException;
/**
* Returns a {@link CompoundFileDirectory} capable of
@ -126,8 +120,8 @@ public abstract class Directory implements Closeable {
* {@link DefaultCompoundFileDirectory}.
* @lucene.experimental
*/
public CompoundFileDirectory openCompoundInput(String name, int bufferSize) throws IOException {
return new DefaultCompoundFileDirectory(this, name, bufferSize, false);
public CompoundFileDirectory openCompoundInput(String name, IOContext context) throws IOException {
return new DefaultCompoundFileDirectory(this, name, context, false);
}
/**
@ -138,8 +132,8 @@ public abstract class Directory implements Closeable {
* {@link DefaultCompoundFileDirectory}.
* @lucene.experimental
*/
public CompoundFileDirectory createCompoundOutput(String name) throws IOException {
return new DefaultCompoundFileDirectory(this, name, 1024, true);
public CompoundFileDirectory createCompoundOutput(String name, IOContext context) throws IOException {
return new DefaultCompoundFileDirectory(this, name, context, true);
}
/** Construct a {@link Lock}.
@ -223,13 +217,13 @@ public abstract class Directory implements Closeable {
* <b>NOTE:</b> this method does not check whether <i>dest<i> exist and will
* overwrite it if it does.
*/
public void copy(Directory to, String src, String dest) throws IOException {
public void copy(Directory to, String src, String dest, IOContext context) throws IOException {
IndexOutput os = null;
IndexInput is = null;
IOException priorException = null;
try {
os = to.createOutput(dest);
is = openInput(src);
os = to.createOutput(dest, context);
is = openInput(src, context);
is.copyBytes(os, is.length());
} catch (IOException ioe) {
priorException = ioe;

View File

@ -122,6 +122,10 @@ public abstract class FSDirectory extends 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
// null means no limite
private Double maxMergeWriteMBPerSec;
private RateLimiter mergeWriteRateLimiter;
// returns the canonical version of the directory, creating it if it doesn't exist.
private static File getCanonicalPath(File file) throws IOException {
return new File(file.getCanonicalPath());
@ -286,11 +290,40 @@ public abstract class FSDirectory extends Directory {
/** Creates an IndexOutput for the file with the given name. */
@Override
public IndexOutput createOutput(String name) throws IOException {
public IndexOutput createOutput(String name, IOContext context) throws IOException {
ensureOpen();
ensureCanWrite(name);
return new FSIndexOutput(this, name);
return new FSIndexOutput(this, name, context.context == IOContext.Context.MERGE ? mergeWriteRateLimiter : null);
}
/** Sets the maximum (approx) MB/sec allowed by all write
* IO performed by merging. Pass null to have no limit.
*
* <p><b>NOTE</b>: if merges are already running there is
* no guarantee this new rate will apply to them; it will
* only apply for certain to new merges.
*
* @lucene.experimental */
public synchronized void setMaxMergeWriteMBPerSec(Double mbPerSec) {
maxMergeWriteMBPerSec = mbPerSec;
if (mbPerSec == null) {
if (mergeWriteRateLimiter != null) {
mergeWriteRateLimiter.setMaxRate(Double.MAX_VALUE);
mergeWriteRateLimiter = null;
}
} else if (mergeWriteRateLimiter != null) {
mergeWriteRateLimiter.setMaxRate(mbPerSec);
} else {
mergeWriteRateLimiter = new RateLimiter(mbPerSec);
}
}
/** See {@link #setMaxMergeWriteMBPerSec}.
*
* @lucene.experimental */
public Double getMaxMergeWriteMBPerSec() {
return maxMergeWriteMBPerSec;
}
protected void ensureCanWrite(String name) throws IOException {
@ -319,13 +352,6 @@ public abstract class FSDirectory extends Directory {
staleFiles.removeAll(toSync);
}
// Inherit javadoc
@Override
public IndexInput openInput(String name) throws IOException {
ensureOpen();
return openInput(name, BufferedIndexInput.BUFFER_SIZE);
}
@Override
public String getLockID() {
ensureOpen();
@ -409,17 +435,22 @@ public abstract class FSDirectory extends Directory {
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 {
private final RateLimiter rateLimiter;
public FSIndexOutput(FSDirectory parent, String name, RateLimiter rateLimiter) throws IOException {
this.parent = parent;
this.name = name;
file = new RandomAccessFile(new File(parent.directory, name), "rw");
isOpen = true;
this.rateLimiter = rateLimiter;
}
/** output methods: */
@Override
public void flushBuffer(byte[] b, int offset, int size) throws IOException {
if (rateLimiter != null) {
rateLimiter.pause(size);
}
file.write(b, offset, size);
}

View File

@ -25,6 +25,7 @@ import java.util.List;
import java.util.Set;
import java.util.HashSet;
/**
* Expert: A Directory instance that switches files between
* two other Directory instances.
@ -125,8 +126,8 @@ public class FileSwitchDirectory extends Directory {
}
@Override
public IndexOutput createOutput(String name) throws IOException {
return getDirectory(name).createOutput(name);
public IndexOutput createOutput(String name, IOContext context) throws IOException {
return getDirectory(name).createOutput(name, context);
}
@Override
@ -145,17 +146,17 @@ public class FileSwitchDirectory extends Directory {
}
@Override
public IndexInput openInput(String name) throws IOException {
return getDirectory(name).openInput(name);
public IndexInput openInput(String name, IOContext context) throws IOException {
return getDirectory(name).openInput(name, context);
}
@Override
public CompoundFileDirectory openCompoundInput(String name, int bufferSize) throws IOException {
return getDirectory(name).openCompoundInput(name, bufferSize);
public CompoundFileDirectory openCompoundInput(String name, IOContext context) throws IOException {
return getDirectory(name).openCompoundInput(name, context);
}
@Override
public CompoundFileDirectory createCompoundOutput(String name) throws IOException {
return getDirectory(name).createCompoundOutput(name);
public CompoundFileDirectory createCompoundOutput(String name, IOContext context) throws IOException {
return getDirectory(name).createCompoundOutput(name, context);
}
}

View File

@ -0,0 +1,77 @@
package org.apache.lucene.store;
/**
* 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.
*/
/**
* <p>A FlushInfo provides information required for a FLUSH context and other optimization operations.
* It is used as part of an {@link IOContext} in case of FLUSH context.</p>
*/
public class FlushInfo {
public final int numDocs;
public final long estimatedSegmentSize;
/**
* <p>Creates a new {@link FlushInfo} instance from
* the values required for a FLUSH {@link IOContext} context.
*
* These values are only estimates and are not the actual values.
*
*/
public FlushInfo(int numDocs, long estimatedSegmentSize) {
this.numDocs = numDocs;
this.estimatedSegmentSize = estimatedSegmentSize;
}
@Override
public int hashCode() {
final int prime = 31;
int result = 1;
result = prime * result
+ (int) (estimatedSegmentSize ^ (estimatedSegmentSize >>> 32));
result = prime * result + numDocs;
return result;
}
@Override
public boolean equals(Object obj) {
if (this == obj)
return true;
if (obj == null)
return false;
if (getClass() != obj.getClass())
return false;
FlushInfo other = (FlushInfo) obj;
if (estimatedSegmentSize != other.estimatedSegmentSize)
return false;
if (numDocs != other.numDocs)
return false;
return true;
}
@Override
public String toString() {
return "FlushInfo [numDocs=" + numDocs + ", estimatedSegmentSize="
+ estimatedSegmentSize + "]";
}
}

View File

@ -0,0 +1,132 @@
package org.apache.lucene.store;
/**
* 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.
*/
/**
* IOContext holds additional details on the merge/search context. A IOContext
* object can never be initialized as null as passed as a parameter to either
* {@link #org.apache.lucene.store.Directory.openInput()} or
* {@link #org.apache.lucene.store.Directory.createInput()}
*/
public class IOContext {
/**
* Context is a enumerator which specifies the context in which the Directory
* is being used for.
*/
public enum Context {
MERGE, READ, FLUSH, DEFAULT
};
/**
* An object of a enumerator Context type
*/
public final Context context;
public final MergeInfo mergeInfo;
public final FlushInfo flushInfo;
public final boolean readOnce;
public static final IOContext DEFAULT = new IOContext(Context.DEFAULT);
public static final IOContext READONCE = new IOContext(true);
public static final IOContext READ = new IOContext(false);
public IOContext() {
this(false);
}
public IOContext(FlushInfo flushInfo) {
assert flushInfo != null;
this.context = Context.FLUSH;
this.mergeInfo = null;
this.readOnce = false;
this.flushInfo = flushInfo;
}
public IOContext(Context context) {
this(context, null);
}
private IOContext(boolean readOnce) {
this.context = Context.READ;
this.mergeInfo = null;
this.readOnce = readOnce;
this.flushInfo = null;
}
public IOContext(MergeInfo mergeInfo) {
this(Context.MERGE, mergeInfo);
}
private IOContext(Context context, MergeInfo mergeInfo) {
assert context != Context.MERGE || mergeInfo != null : "MergeInfo must not be null if context is MERGE";
assert context != Context.FLUSH : "Use IOContext(FlushInfo) to create a FLUSH IOContext";
this.context = context;
this.readOnce = false;
this.mergeInfo = mergeInfo;
this.flushInfo = null;
}
@Override
public int hashCode() {
final int prime = 31;
int result = 1;
result = prime * result + ((context == null) ? 0 : context.hashCode());
result = prime * result + ((flushInfo == null) ? 0 : flushInfo.hashCode());
result = prime * result + ((mergeInfo == null) ? 0 : mergeInfo.hashCode());
result = prime * result + (readOnce ? 1231 : 1237);
return result;
}
@Override
public boolean equals(Object obj) {
if (this == obj)
return true;
if (obj == null)
return false;
if (getClass() != obj.getClass())
return false;
IOContext other = (IOContext) obj;
if (context != other.context)
return false;
if (flushInfo == null) {
if (other.flushInfo != null)
return false;
} else if (!flushInfo.equals(other.flushInfo))
return false;
if (mergeInfo == null) {
if (other.mergeInfo != null)
return false;
} else if (!mergeInfo.equals(other.mergeInfo))
return false;
if (readOnce != other.readOnce)
return false;
return true;
}
@Override
public String toString() {
return "IOContext [context=" + context + ", mergeInfo=" + mergeInfo
+ ", flushInfo=" + flushInfo + ", readOnce=" + readOnce + "]";
}
}

View File

@ -209,7 +209,7 @@ public class MMapDirectory extends FSDirectory {
/** Creates an IndexInput for the file with the given name. */
@Override
public IndexInput openInput(String name, int bufferSize) throws IOException {
public IndexInput openInput(String name, IOContext context) throws IOException {
ensureOpen();
File f = new File(getDirectory(), name);
RandomAccessFile raf = new RandomAccessFile(f, "r");
@ -221,15 +221,15 @@ public class MMapDirectory extends FSDirectory {
}
@Override
public CompoundFileDirectory openCompoundInput(String name, int bufferSize) throws IOException {
return new MMapCompoundFileDirectory(name, bufferSize);
public CompoundFileDirectory openCompoundInput(String name, IOContext context) throws IOException {
return new MMapCompoundFileDirectory(name, context);
}
private final class MMapCompoundFileDirectory extends CompoundFileDirectory {
private RandomAccessFile raf = null;
public MMapCompoundFileDirectory(String fileName, int readBufferSize) throws IOException {
super(MMapDirectory.this, fileName, readBufferSize);
public MMapCompoundFileDirectory(String fileName, IOContext context) throws IOException {
super(MMapDirectory.this, fileName, context);
IndexInput stream = null;
try {
File f = new File(MMapDirectory.this.getDirectory(), fileName);
@ -438,4 +438,5 @@ public class MMapDirectory extends FSDirectory {
}
}
}
}

View File

@ -0,0 +1,89 @@
package org.apache.lucene.store;
/**
* 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.
*/
/**
* <p>A MergeInfo provides information required for a MERGE context and other optimization operations.
* It is used as part of an {@link IOContext} in case of MERGE context.</p>
*/
public class MergeInfo {
public final int totalDocCount;
public final long estimatedMergeBytes;
public final boolean isExternal;
public final boolean optimize;
/**
* <p>Creates a new {@link MergeInfo} instance from
* the values required for a MERGE {@link IOContext} context.
*
* These values are only estimates and are not the actual values.
*
*/
public MergeInfo(int totalDocCount, long estimatedMergeBytes, boolean isExternal, boolean optimize) {
this.totalDocCount = totalDocCount;
this.estimatedMergeBytes = estimatedMergeBytes;
this.isExternal = isExternal;
this.optimize = optimize;
}
@Override
public int hashCode() {
final int prime = 31;
int result = 1;
result = prime * result
+ (int) (estimatedMergeBytes ^ (estimatedMergeBytes >>> 32));
result = prime * result + (isExternal ? 1231 : 1237);
result = prime * result + (optimize ? 1231 : 1237);
result = prime * result + totalDocCount;
return result;
}
@Override
public boolean equals(Object obj) {
if (this == obj)
return true;
if (obj == null)
return false;
if (getClass() != obj.getClass())
return false;
MergeInfo other = (MergeInfo) obj;
if (estimatedMergeBytes != other.estimatedMergeBytes)
return false;
if (isExternal != other.isExternal)
return false;
if (optimize != other.optimize)
return false;
if (totalDocCount != other.totalDocCount)
return false;
return true;
}
@Override
public String toString() {
return "MergeInfo [totalDocCount=" + totalDocCount
+ ", estimatedMergeBytes=" + estimatedMergeBytes + ", isExternal="
+ isExternal + ", optimize=" + optimize + "]";
}
}

View File

@ -76,22 +76,22 @@ public class NIOFSDirectory extends FSDirectory {
/** Creates an IndexInput for the file with the given name. */
@Override
public IndexInput openInput(String name, int bufferSize) throws IOException {
public IndexInput openInput(String name, IOContext context) throws IOException {
ensureOpen();
return new NIOFSIndexInput(new File(getDirectory(), name), bufferSize, getReadChunkSize());
return new NIOFSIndexInput(new File(getDirectory(), name), context, getReadChunkSize());
}
@Override
public CompoundFileDirectory openCompoundInput(String name, int bufferSize) throws IOException {
return new NIOFSCompoundFileDirectory(name, bufferSize);
public CompoundFileDirectory openCompoundInput(String name, IOContext context) throws IOException {
return new NIOFSCompoundFileDirectory(name, context);
}
private final class NIOFSCompoundFileDirectory extends CompoundFileDirectory {
private SimpleFSIndexInput.Descriptor fd;
private FileChannel fc;
public NIOFSCompoundFileDirectory(String fileName, int readBufferSize) throws IOException {
super(NIOFSDirectory.this, fileName, readBufferSize);
public NIOFSCompoundFileDirectory(String fileName, IOContext context) throws IOException {
super(NIOFSDirectory.this, fileName, context);
IndexInput stream = null;
try {
File f = new File(NIOFSDirectory.this.getDirectory(), fileName);
@ -131,8 +131,8 @@ public class NIOFSDirectory extends FSDirectory {
final FileChannel channel;
public NIOFSIndexInput(File path, int bufferSize, int chunkSize) throws IOException {
super(path, bufferSize, chunkSize);
public NIOFSIndexInput(File path, IOContext context, int chunkSize) throws IOException {
super(path, context, chunkSize);
channel = file.getChannel();
}
@ -229,4 +229,5 @@ public class NIOFSDirectory extends FSDirectory {
}
}
}
}

View File

@ -27,6 +27,7 @@ import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicLong;
/**
* A memory-resident {@link Directory} implementation. Locking
* implementation is by default the {@link SingleInstanceLockFactory}
@ -38,7 +39,7 @@ public class RAMDirectory extends Directory {
// *****
// Lock acquisition sequence: RAMDirectory, then RAMFile
// *****
// *****
/** Constructs an empty {@link Directory}. */
public RAMDirectory() {
@ -65,14 +66,14 @@ public class RAMDirectory extends Directory {
* @param dir a <code>Directory</code> value
* @exception IOException if an error occurs
*/
public RAMDirectory(Directory dir) throws IOException {
this(dir, false);
public RAMDirectory(Directory dir, IOContext context) throws IOException {
this(dir, false, context);
}
private RAMDirectory(Directory dir, boolean closeDir) throws IOException {
private RAMDirectory(Directory dir, boolean closeDir, IOContext context) throws IOException {
this();
for (String file : dir.listAll()) {
dir.copy(this, file, file);
dir.copy(this, file, file, context);
}
if (closeDir) {
dir.close();
@ -149,7 +150,7 @@ public class RAMDirectory extends Directory {
/** Creates a new, empty file in the directory with the given name. Returns a stream writing this file. */
@Override
public IndexOutput createOutput(String name) throws IOException {
public IndexOutput createOutput(String name, IOContext context) throws IOException {
ensureOpen();
RAMFile file = newRAMFile();
RAMFile existing = fileMap.remove(name);
@ -176,7 +177,7 @@ public class RAMDirectory extends Directory {
/** Returns a stream reading an existing file. */
@Override
public IndexInput openInput(String name) throws IOException {
public IndexInput openInput(String name, IOContext context) throws IOException {
ensureOpen();
RAMFile file = fileMap.get(name);
if (file == null) {

View File

@ -0,0 +1,78 @@
package org.apache.lucene.store;
/**
* 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.
*/
import org.apache.lucene.util.ThreadInterruptedException;
/** Simple class to rate limit IO. Typically it's shared
* across multiple IndexInputs or IndexOutputs (for example
* those involved all merging). Those IndexInputs and
* IndexOutputs would call {@link #pause} whenever they
* want to read bytes or write bytes. */
public class RateLimiter {
private volatile double nsPerByte;
private volatile long lastNS;
// TODO: we could also allow eg a sub class to dynamically
// determine the allowed rate, eg if an app wants to
// change the allowed rate over time or something
/** mbPerSec is the MB/sec max IO rate */
public RateLimiter(double mbPerSec) {
setMaxRate(mbPerSec);
}
public void setMaxRate(double mbPerSec) {
nsPerByte = 1000000000. / (1024*1024*mbPerSec);
}
/** Pauses, if necessary, to keep the instantaneous IO
* rate at or below the target. NOTE: multiple threads
* may safely use this, however the implementation is
* not perfectly thread safe but likely in practice this
* is harmless (just means in some rare cases the rate
* might exceed the target). It's best to call this
* with a biggish count, not one byte at a time. */
public void pause(long bytes) {
// TODO: this is purely instantenous rate; maybe we
// should also offer decayed recent history one?
final long targetNS = lastNS = lastNS + ((long) (bytes * nsPerByte));
long curNS = System.nanoTime();
if (lastNS < curNS) {
lastNS = curNS;
}
// While loop because Thread.sleep doesn't alway sleep
// enough:
while(true) {
final long pauseNS = targetNS - curNS;
if (pauseNS > 0) {
try {
Thread.sleep((int) (pauseNS/1000000), (int) (pauseNS % 1000000));
} catch (InterruptedException ie) {
throw new ThreadInterruptedException(ie);
}
curNS = System.nanoTime();
continue;
}
break;
}
}
}

View File

@ -23,6 +23,7 @@ import java.io.RandomAccessFile;
import org.apache.lucene.util.IOUtils;
/** A straightforward implementation of {@link FSDirectory}
* using java.io.RandomAccessFile. However, this class has
* poor concurrent performance (multiple threads will
@ -53,21 +54,21 @@ public class SimpleFSDirectory extends FSDirectory {
/** Creates an IndexInput for the file with the given name. */
@Override
public IndexInput openInput(String name, int bufferSize) throws IOException {
public IndexInput openInput(String name, IOContext context) throws IOException {
ensureOpen();
return new SimpleFSIndexInput(new File(directory, name), bufferSize, getReadChunkSize());
return new SimpleFSIndexInput(new File(directory, name), context, getReadChunkSize());
}
@Override
public CompoundFileDirectory openCompoundInput(String name, int bufferSize) throws IOException {
return new SimpleFSCompoundFileDirectory(name, bufferSize);
public CompoundFileDirectory openCompoundInput(String name, IOContext context) throws IOException {
return new SimpleFSCompoundFileDirectory(name, context);
}
private final class SimpleFSCompoundFileDirectory extends CompoundFileDirectory {
private SimpleFSIndexInput.Descriptor fd;
public SimpleFSCompoundFileDirectory(String fileName, int readBufferSize) throws IOException {
super(SimpleFSDirectory.this, fileName, readBufferSize);
public SimpleFSCompoundFileDirectory(String fileName, IOContext context) throws IOException {
super(SimpleFSDirectory.this, fileName, context);
IndexInput stream = null;
try {
final File f = new File(SimpleFSDirectory.this.getDirectory(), fileName);
@ -128,8 +129,8 @@ public class SimpleFSDirectory extends FSDirectory {
protected final long off;
protected final long end;
public SimpleFSIndexInput(File path, int bufferSize, int chunkSize) throws IOException {
super(bufferSize);
public SimpleFSIndexInput(File path, IOContext context, int chunkSize) throws IOException {
super(context);
this.file = new Descriptor(path, "r");
this.chunkSize = chunkSize;
this.off = 0L;

View File

@ -20,9 +20,12 @@ package org.apache.lucene.util;
import java.io.IOException;
import java.util.Arrays;
import org.apache.lucene.index.MergePolicy.OneMerge;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.IOContext.Context;
/** Optimized implementation of a vector of bits. This is more-or-less like
* java.util.BitSet, but also includes the following:
@ -215,8 +218,8 @@ public final class BitVector implements Cloneable, Bits {
/** Writes this vector to the file <code>name</code> in Directory
<code>d</code>, in a format that can be read by the constructor {@link
#BitVector(Directory, String)}. */
public final void write(Directory d, String name) throws IOException {
IndexOutput output = d.createOutput(name);
public final void write(Directory d, String name, IOContext context) throws IOException {
IndexOutput output = d.createOutput(name, context);
try {
output.writeInt(-2);
CodecUtil.writeHeader(output, CODEC, VERSION_CURRENT);
@ -328,8 +331,8 @@ public final class BitVector implements Cloneable, Bits {
/** Constructs a bit vector from the file <code>name</code> in Directory
<code>d</code>, as written by the {@link #write} method.
*/
public BitVector(Directory d, String name) throws IOException {
IndexInput input = d.openInput(name);
public BitVector(Directory d, String name, IOContext context) throws IOException {
IndexInput input = d.openInput(name, context);
try {
final int firstInt = input.readInt();

View File

@ -84,8 +84,8 @@ public class MockFixedIntBlockCodec extends Codec {
}
@Override
public IntIndexInput openInput(Directory dir, String fileName, int readBufferSize) throws IOException {
return new FixedIntBlockIndexInput(dir.openInput(fileName, readBufferSize)) {
public IntIndexInput openInput(Directory dir, String fileName, IOContext context) throws IOException {
return new FixedIntBlockIndexInput(dir.openInput(fileName, context)) {
@Override
protected BlockReader getBlockReader(final IndexInput in, final int[] buffer) throws IOException {
@ -102,8 +102,8 @@ public class MockFixedIntBlockCodec extends Codec {
}
@Override
public IntIndexOutput createOutput(Directory dir, String fileName) throws IOException {
IndexOutput out = dir.createOutput(fileName);
public IntIndexOutput createOutput(Directory dir, String fileName, IOContext context) throws IOException {
IndexOutput out = dir.createOutput(fileName, context);
boolean success = false;
try {
FixedIntBlockIndexOutput ret = new FixedIntBlockIndexOutput(out, blockSize) {
@ -160,7 +160,7 @@ public class MockFixedIntBlockCodec extends Codec {
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postingsReader = new SepPostingsReaderImpl(state.dir,
state.segmentInfo,
state.readBufferSize,
state.context,
new MockIntFactory(blockSize), state.codecId);
TermsIndexReaderBase indexReader;
@ -170,7 +170,8 @@ public class MockFixedIntBlockCodec extends Codec {
state.fieldInfos,
state.segmentInfo.name,
state.termsIndexDivisor,
BytesRef.getUTF8SortedAsUnicodeComparator(), state.codecId);
BytesRef.getUTF8SortedAsUnicodeComparator(), state.codecId,
IOContext.DEFAULT);
success = true;
} finally {
if (!success) {
@ -185,7 +186,7 @@ public class MockFixedIntBlockCodec extends Codec {
state.fieldInfos,
state.segmentInfo.name,
postingsReader,
state.readBufferSize,
state.context,
StandardCodec.TERMS_CACHE_SIZE,
state.codecId);
success = true;
@ -224,6 +225,6 @@ public class MockFixedIntBlockCodec extends Codec {
@Override
public PerDocValues docsProducer(SegmentReadState state) throws IOException {
return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId, getDocValuesUseCFS(), getDocValuesSortComparator());
return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId, getDocValuesUseCFS(), getDocValuesSortComparator(), state.context);
}
}

View File

@ -48,6 +48,7 @@ import org.apache.lucene.index.codecs.BlockTermsWriter;
import org.apache.lucene.index.codecs.TermsIndexReaderBase;
import org.apache.lucene.index.codecs.TermsIndexWriterBase;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
@ -82,8 +83,8 @@ public class MockVariableIntBlockCodec extends Codec {
}
@Override
public IntIndexInput openInput(Directory dir, String fileName, int readBufferSize) throws IOException {
final IndexInput in = dir.openInput(fileName, readBufferSize);
public IntIndexInput openInput(Directory dir, String fileName, IOContext context) throws IOException {
final IndexInput in = dir.openInput(fileName, context);
final int baseBlockSize = in.readInt();
return new VariableIntBlockIndexInput(in) {
@ -106,8 +107,8 @@ public class MockVariableIntBlockCodec extends Codec {
}
@Override
public IntIndexOutput createOutput(Directory dir, String fileName) throws IOException {
final IndexOutput out = dir.createOutput(fileName);
public IntIndexOutput createOutput(Directory dir, String fileName, IOContext context) throws IOException {
final IndexOutput out = dir.createOutput(fileName, context);
boolean success = false;
try {
out.writeInt(baseBlockSize);
@ -182,7 +183,7 @@ public class MockVariableIntBlockCodec extends Codec {
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postingsReader = new SepPostingsReaderImpl(state.dir,
state.segmentInfo,
state.readBufferSize,
state.context,
new MockIntFactory(baseBlockSize), state.codecId);
TermsIndexReaderBase indexReader;
@ -193,7 +194,7 @@ public class MockVariableIntBlockCodec extends Codec {
state.segmentInfo.name,
state.termsIndexDivisor,
BytesRef.getUTF8SortedAsUnicodeComparator(),
state.codecId);
state.codecId, state.context);
success = true;
} finally {
if (!success) {
@ -208,7 +209,7 @@ public class MockVariableIntBlockCodec extends Codec {
state.fieldInfos,
state.segmentInfo.name,
postingsReader,
state.readBufferSize,
state.context,
StandardCodec.TERMS_CACHE_SIZE,
state.codecId);
success = true;
@ -247,6 +248,6 @@ public class MockVariableIntBlockCodec extends Codec {
@Override
public PerDocValues docsProducer(SegmentReadState state) throws IOException {
return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId, getDocValuesUseCFS(), getDocValuesSortComparator());
return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId, getDocValuesUseCFS(), getDocValuesSortComparator(), state.context);
}
}

View File

@ -61,6 +61,7 @@ import org.apache.lucene.index.codecs.sep.SepPostingsWriterImpl;
import org.apache.lucene.index.codecs.standard.StandardPostingsReader;
import org.apache.lucene.index.codecs.standard.StandardPostingsWriter;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
@ -103,23 +104,23 @@ public class MockRandomCodec extends Codec {
}
@Override
public IntIndexInput openInput(Directory dir, String fileName, int readBufferSize) throws IOException {
public IntIndexInput openInput(Directory dir, String fileName, IOContext context) throws IOException {
// Must only use extension, because IW.addIndexes can
// rename segment!
final IntStreamFactory f = delegates.get((Math.abs(salt ^ getExtension(fileName).hashCode())) % delegates.size());
if (LuceneTestCase.VERBOSE) {
System.out.println("MockRandomCodec: read using int factory " + f + " from fileName=" + fileName);
}
return f.openInput(dir, fileName, readBufferSize);
return f.openInput(dir, fileName, context);
}
@Override
public IntIndexOutput createOutput(Directory dir, String fileName) throws IOException {
public IntIndexOutput createOutput(Directory dir, String fileName, IOContext context) throws IOException {
final IntStreamFactory f = delegates.get((Math.abs(salt ^ getExtension(fileName).hashCode())) % delegates.size());
if (LuceneTestCase.VERBOSE) {
System.out.println("MockRandomCodec: write using int factory " + f + " to fileName=" + fileName);
}
return f.createOutput(dir, fileName);
return f.createOutput(dir, fileName, context);
}
}
@ -140,7 +141,7 @@ public class MockRandomCodec extends Codec {
}
final String seedFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, SEED_EXT);
final IndexOutput out = state.directory.createOutput(seedFileName);
final IndexOutput out = state.directory.createOutput(seedFileName, state.context);
try {
out.writeLong(seed);
} finally {
@ -241,7 +242,7 @@ public class MockRandomCodec extends Codec {
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
final String seedFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.codecId, SEED_EXT);
final IndexInput in = state.dir.openInput(seedFileName);
final IndexInput in = state.dir.openInput(seedFileName, state.context);
final long seed = in.readLong();
if (LuceneTestCase.VERBOSE) {
System.out.println("MockRandomCodec: reading from seg=" + state.segmentInfo.name + " seed=" + seed);
@ -259,12 +260,12 @@ public class MockRandomCodec extends Codec {
if (random.nextBoolean()) {
postingsReader = new SepPostingsReaderImpl(state.dir, state.segmentInfo,
readBufferSize, new MockIntStreamFactory(random), state.codecId);
state.context, new MockIntStreamFactory(random), state.codecId);
} else {
if (LuceneTestCase.VERBOSE) {
System.out.println("MockRandomCodec: reading Standard postings");
}
postingsReader = new StandardPostingsReader(state.dir, state.segmentInfo, readBufferSize, state.codecId);
postingsReader = new StandardPostingsReader(state.dir, state.segmentInfo, state.context, state.codecId);
}
if (random.nextBoolean()) {
@ -293,7 +294,7 @@ public class MockRandomCodec extends Codec {
state.segmentInfo.name,
state.termsIndexDivisor,
BytesRef.getUTF8SortedAsUnicodeComparator(),
state.codecId);
state.codecId, state.context);
} else {
final int n2 = random.nextInt(3);
if (n2 == 1) {
@ -311,7 +312,7 @@ public class MockRandomCodec extends Codec {
state.fieldInfos,
state.segmentInfo.name,
state.termsIndexDivisor,
state.codecId);
state.codecId, state.context);
}
success = true;
} finally {
@ -329,7 +330,7 @@ public class MockRandomCodec extends Codec {
state.fieldInfos,
state.segmentInfo.name,
postingsReader,
readBufferSize,
state.context,
termsCacheSize,
state.codecId);
success = true;
@ -385,6 +386,6 @@ public class MockRandomCodec extends Codec {
@Override
public PerDocValues docsProducer(SegmentReadState state) throws IOException {
return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId, getDocValuesUseCFS(), getDocValuesSortComparator());
return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId, getDocValuesUseCFS(), getDocValuesSortComparator(), state.context);
}
}

View File

@ -93,7 +93,7 @@ public class MockSepCodec extends Codec {
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postingsReader = new SepPostingsReaderImpl(state.dir, state.segmentInfo,
state.readBufferSize, new MockSingleIntFactory(), state.codecId);
state.context, new MockSingleIntFactory(), state.codecId);
TermsIndexReaderBase indexReader;
boolean success = false;
@ -103,7 +103,7 @@ public class MockSepCodec extends Codec {
state.segmentInfo.name,
state.termsIndexDivisor,
BytesRef.getUTF8SortedAsUnicodeComparator(),
state.codecId);
state.codecId, state.context);
success = true;
} finally {
if (!success) {
@ -118,7 +118,7 @@ public class MockSepCodec extends Codec {
state.fieldInfos,
state.segmentInfo.name,
postingsReader,
state.readBufferSize,
state.context,
StandardCodec.TERMS_CACHE_SIZE,
state.codecId);
success = true;
@ -161,6 +161,6 @@ public class MockSepCodec extends Codec {
@Override
public PerDocValues docsProducer(SegmentReadState state) throws IOException {
return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId, getDocValuesUseCFS(), getDocValuesSortComparator());
return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId, getDocValuesUseCFS(), getDocValuesSortComparator(), state.context);
}
}

View File

@ -18,6 +18,7 @@ package org.apache.lucene.index.codecs.mocksep;
*/
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.index.codecs.sep.IntStreamFactory;
import org.apache.lucene.index.codecs.sep.IntIndexInput;
import org.apache.lucene.index.codecs.sep.IntIndexOutput;
@ -27,11 +28,11 @@ import java.io.IOException;
/** @lucene.experimental */
public class MockSingleIntFactory extends IntStreamFactory {
@Override
public IntIndexInput openInput(Directory dir, String fileName, int readBufferSize) throws IOException {
return new MockSingleIntIndexInput(dir, fileName, readBufferSize);
public IntIndexInput openInput(Directory dir, String fileName, IOContext context) throws IOException {
return new MockSingleIntIndexInput(dir, fileName, context);
}
@Override
public IntIndexOutput createOutput(Directory dir, String fileName) throws IOException {
return new MockSingleIntIndexOutput(dir, fileName);
public IntIndexOutput createOutput(Directory dir, String fileName, IOContext context) throws IOException {
return new MockSingleIntIndexOutput(dir, fileName, context);
}
}

View File

@ -22,6 +22,7 @@ import java.io.IOException;
import org.apache.lucene.index.codecs.sep.IntIndexInput;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.CodecUtil;
@ -35,9 +36,9 @@ import org.apache.lucene.util.CodecUtil;
public class MockSingleIntIndexInput extends IntIndexInput {
private final IndexInput in;
public MockSingleIntIndexInput(Directory dir, String fileName, int readBufferSize)
public MockSingleIntIndexInput(Directory dir, String fileName, IOContext context)
throws IOException {
in = dir.openInput(fileName, readBufferSize);
in = dir.openInput(fileName, context);
CodecUtil.checkHeader(in, MockSingleIntIndexOutput.CODEC,
MockSingleIntIndexOutput.VERSION_START,
MockSingleIntIndexOutput.VERSION_START);

View File

@ -17,6 +17,7 @@ package org.apache.lucene.index.codecs.mocksep;
* limitations under the License.
*/
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.CodecUtil;
@ -35,8 +36,8 @@ public class MockSingleIntIndexOutput extends IntIndexOutput {
final static int VERSION_START = 0;
final static int VERSION_CURRENT = VERSION_START;
public MockSingleIntIndexOutput(Directory dir, String fileName) throws IOException {
out = dir.createOutput(fileName);
public MockSingleIntIndexOutput(Directory dir, String fileName, IOContext context) throws IOException {
out = dir.createOutput(fileName, context);
boolean success = false;
try {
CodecUtil.writeHeader(out, CODEC, VERSION_CURRENT);

View File

@ -50,12 +50,12 @@ class PreFlexFieldsWriter extends FieldsConsumer {
state.termIndexInterval);
final String freqFile = IndexFileNames.segmentFileName(state.segmentName, "", PreFlexCodec.FREQ_EXTENSION);
freqOut = state.directory.createOutput(freqFile);
freqOut = state.directory.createOutput(freqFile, state.context);
totalNumDocs = state.numDocs;
if (state.fieldInfos.hasProx()) {
final String proxFile = IndexFileNames.segmentFileName(state.segmentName, "", PreFlexCodec.PROX_EXTENSION);
proxOut = state.directory.createOutput(proxFile);
proxOut = state.directory.createOutput(proxFile, state.context);
} else {
proxOut = null;
}

View File

@ -50,7 +50,7 @@ public class PreFlexRWCodec extends PreFlexCodec {
// Whenever IW opens readers, eg for merging, we have to
// keep terms order in UTF16:
return new PreFlexFields(state.dir, state.fieldInfos, state.segmentInfo, state.readBufferSize, state.termsIndexDivisor) {
return new PreFlexFields(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.termsIndexDivisor) {
@Override
protected boolean sortTermsByUnicode() {
// We carefully peek into stack track above us: if

View File

@ -26,6 +26,7 @@ import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.codecs.preflex.PreFlexCodec;
import org.apache.lucene.index.codecs.preflex.TermInfo;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CharsRef;
@ -125,7 +126,7 @@ final class TermInfosWriter implements Closeable {
isIndex = isi;
output = directory.createOutput(IndexFileNames.segmentFileName(segment, "",
(isIndex ? PreFlexCodec.TERMS_INDEX_EXTENSION
: PreFlexCodec.TERMS_EXTENSION)));
: PreFlexCodec.TERMS_EXTENSION)), IOContext.DEFAULT);
boolean success = false;
try {
output.writeInt(FORMAT_CURRENT); // write format

Some files were not shown because too many files have changed in this diff Show More