LUCENE-2793: initial commit from patch

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/LUCENE2793@1137639 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Simon Willnauer 2011-06-20 14:28:24 +00:00
parent 9deba20d65
commit 92b07b725e
130 changed files with 746 additions and 524 deletions

View File

@ -94,11 +94,12 @@ 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;
try {
//nocommit we should pass the IOContext down to the TermIndexReader
indexReader = new AppendingTermsIndexReader(state.dir,
state.fieldInfos,
state.segmentInfo.name,
@ -116,7 +117,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;

View File

@ -20,6 +20,7 @@ package org.apache.lucene.index.codecs.appending;
import java.io.IOException;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.IOContext;
import org.apache.lucene.index.codecs.DefaultSegmentInfosReader;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IndexInput;
@ -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

@ -19,6 +19,7 @@ package org.apache.lucene.index.codecs.appending;
import java.io.IOException;
import org.apache.lucene.index.IOContext;
import org.apache.lucene.index.codecs.DefaultSegmentInfosWriter;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IndexOutput;
@ -26,9 +27,9 @@ 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

@ -20,6 +20,7 @@ package org.apache.lucene.index.codecs.appending;
import java.io.IOException;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IOContext;
import org.apache.lucene.index.codecs.PostingsReaderBase;
import org.apache.lucene.index.codecs.BlockTermsReader;
import org.apache.lucene.index.codecs.BlockTermsWriter;
@ -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

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

View File

@ -25,6 +25,7 @@ import java.io.FileOutputStream;
import java.nio.ByteBuffer;
import java.nio.channels.FileChannel;
import org.apache.lucene.index.IOContext;
import org.apache.lucene.store.Directory; // javadoc
import org.apache.lucene.store.NativeFSLockFactory; // javadoc
@ -69,15 +70,17 @@ 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);
//nocommit - use buffer based on IOContext
return new DirectIOLinuxIndexInput(new File(getDirectory(), name), forcedBufferSize == 0 ? BufferedIndexInput.BUFFER_SIZE : forcedBufferSize);
}
@Override
public IndexOutput createOutput(String name) throws IOException {
public IndexOutput createOutput(String name,IOContext context) throws IOException {
ensureOpen();
ensureCanWrite(name);
//nocommit - use buffer based on IOContext
return new DirectIOLinuxIndexOutput(new File(getDirectory(), name), forcedBufferSize == 0 ? BufferedIndexOutput.BUFFER_SIZE : forcedBufferSize);
}

View File

@ -24,8 +24,10 @@ import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.lucene.index.ConcurrentMergeScheduler;
import org.apache.lucene.index.IOContext;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexWriter; // javadocs
import org.apache.lucene.index.MergeInfo;
import org.apache.lucene.index.MergePolicy;
import org.apache.lucene.index.MergeScheduler;
import org.apache.lucene.store.RAMDirectory; // javadocs
@ -193,17 +195,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 +221,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,18 +229,9 @@ 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);
}
}
@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);
return delegate.openInput(name, context);
}
}
@ -271,18 +264,19 @@ public class NRTCachingDirectory extends Directory {
/** 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;
}
@ -291,7 +285,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,8 @@ package org.apache.lucene.store;
import java.io.File;
import java.io.IOException;
import org.apache.lucene.index.IOContext;
import org.apache.lucene.store.Directory; // javadoc
import org.apache.lucene.store.NativeFSLockFactory; // javadoc
@ -67,9 +69,10 @@ 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));
//nocommit - use buffer based on IOContext
return new WindowsIndexInput(new File(getDirectory(), name), DEFAULT_BUFFERSIZE);
}
protected static class WindowsIndexInput extends BufferedIndexInput {

View File

@ -27,6 +27,7 @@ import org.apache.lucene.document.Field.Store;
import org.apache.lucene.document.Field.TermVector;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.IOContext;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
@ -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

@ -24,6 +24,7 @@ import java.util.List;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.index.IOContext;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
@ -114,7 +115,7 @@ public class TestNRTCachingDirectory extends LuceneTestCase {
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

@ -224,7 +224,8 @@ class BufferedDeletesStream {
// Lock order: IW -> BD -> RP
assert readerPool.infoIsLive(info);
final SegmentReader reader = readerPool.get(info, false);
//nocommit is IOContext.DEFAULT the right thing to do here?
final SegmentReader reader = readerPool.get(info, false, IOContext.DEFAULT);
int delCount = 0;
final boolean segAllDeletes;
try {
@ -273,7 +274,8 @@ class BufferedDeletesStream {
if (coalescedDeletes != null) {
// Lock order: IW -> BD -> RP
assert readerPool.infoIsLive(info);
SegmentReader reader = readerPool.get(info, false);
//nocommit is IOContext.DEFAULT the right thing to do here?
SegmentReader reader = readerPool.get(info, false, IOContext.DEFAULT);
int delCount = 0;
final boolean segAllDeletes;
try {

View File

@ -25,6 +25,8 @@ import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.document.AbstractField; // for javadocs
import org.apache.lucene.document.Document;
import org.apache.lucene.index.IOContext.Context;
import org.apache.lucene.index.MergePolicy.OneMerge;
import org.apache.lucene.index.codecs.CodecProvider;
import org.apache.lucene.index.codecs.DefaultSegmentInfosWriter;
import org.apache.lucene.index.codecs.PerDocValues;
@ -342,7 +344,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)
@ -479,7 +481,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

@ -36,8 +36,6 @@ import java.io.IOException;
*/
public class CompoundFileReader extends Directory {
private int readBufferSize;
private static final class FileEntry {
long offset;
long length;
@ -49,21 +47,16 @@ public class CompoundFileReader extends Directory {
private IndexInput stream;
private HashMap<String,FileEntry> entries = new HashMap<String,FileEntry>();
public CompoundFileReader(Directory dir, String name) throws IOException {
this(dir, name, BufferedIndexInput.BUFFER_SIZE);
}
public CompoundFileReader(Directory dir, String name, int readBufferSize) throws IOException {
public CompoundFileReader(Directory dir, String name, IOContext context) throws IOException {
assert !(dir instanceof CompoundFileReader) : "compound file inside of compound file: " + name;
directory = dir;
fileName = name;
this.readBufferSize = readBufferSize;
boolean success = false;
try {
stream = dir.openInput(name, readBufferSize);
stream = dir.openInput(name, context);
// read the first VInt. If it is negative, it's the version number
// otherwise it's the count (pre-3.1 indexes)
@ -141,13 +134,7 @@ public class CompoundFileReader 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 {
if (stream == null)
throw new IOException("Stream closed");
@ -155,8 +142,8 @@ public class CompoundFileReader extends Directory {
final FileEntry entry = entries.get(id);
if (entry == null)
throw new IOException("No sub-file with id " + id + " found (files: " + entries.keySet() + ")");
return new CSIndexInput(stream, entry.offset, entry.length, readBufferSize);
// nocommit set read buffer size based on IOContext
return new CSIndexInput(stream, entry.offset, entry.length, BufferedIndexInput.BUFFER_SIZE);
}
/** Returns an array of strings, one for each file in the directory. */
@ -209,7 +196,7 @@ public class CompoundFileReader extends Directory {
/** Not implemented
* @throws UnsupportedOperationException */
@Override
public IndexOutput createOutput(String name) {
public IndexOutput createOutput(String name, IOContext context) {
throw new UnsupportedOperationException();
}

View File

@ -76,22 +76,23 @@ public final class CompoundFileWriter {
// switch to a new format!
static final int FORMAT_CURRENT = FORMAT_NO_SEGMENT_PREFIX;
private Directory directory;
private String fileName;
private HashSet<String> ids;
private LinkedList<FileEntry> entries;
private final Directory directory;
private final String fileName;
private final IOContext context;
private final HashSet<String> ids;
private final LinkedList<FileEntry> entries;
private boolean merged = false;
private MergeState.CheckAbort checkAbort;
private final MergeState.CheckAbort checkAbort;
/** Create the compound stream in the specified file. The file name is the
* entire name (no extensions are added).
* @throws NullPointerException if <code>dir</code> or <code>name</code> is null
*/
public CompoundFileWriter(Directory dir, String name) {
this(dir, name, null);
public CompoundFileWriter(Directory dir, String name, IOContext context) {
this(dir, name, context, null);
}
CompoundFileWriter(Directory dir, String name, MergeState.CheckAbort checkAbort) {
CompoundFileWriter(Directory dir, String name, IOContext context, MergeState.CheckAbort checkAbort) {
if (dir == null)
throw new NullPointerException("directory cannot be null");
if (name == null)
@ -101,6 +102,7 @@ public final class CompoundFileWriter {
fileName = name;
ids = new HashSet<String>();
entries = new LinkedList<FileEntry>();
this.context = context;
}
/** Returns the directory of the compound file. */
@ -164,7 +166,7 @@ public final class CompoundFileWriter {
merged = true;
// open the compound stream
IndexOutput os = directory.createOutput(fileName);
IndexOutput os = directory.createOutput(fileName, context);
IOException priorException = null;
try {
// Write the Version info - must be a VInt because CFR reads a VInt
@ -228,7 +230,7 @@ public final class CompoundFileWriter {
* output stream.
*/
private void copyFile(FileEntry source, IndexOutput os) throws IOException {
IndexInput is = source.dir.openInput(source.file);
IndexInput is = source.dir.openInput(source.file, context);
try {
long startPtr = os.getFilePointer();
long length = is.length();

View File

@ -34,6 +34,7 @@ import org.apache.lucene.document.FieldSelector;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.Lock;
import org.apache.lucene.store.LockObtainFailedException;
import org.apache.lucene.index.IOContext.Context;
import org.apache.lucene.index.codecs.CodecProvider;
import org.apache.lucene.index.codecs.PerDocValues;
import org.apache.lucene.util.Bits;
@ -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

@ -28,6 +28,7 @@ import java.util.concurrent.atomic.AtomicLong;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.index.DocumentsWriterDeleteQueue.DeleteSlice;
import org.apache.lucene.index.IOContext.Context;
import org.apache.lucene.search.SimilarityProvider;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BitVector;
@ -428,7 +429,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(Context.FLUSH));
final double startMBUsed = parent.flushControl.netBytes() / 1024. / 1024.;
// Apply delete-by-docID now (delete-byDocID only
// happens when an exception is hit processing that

View File

@ -271,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 {
@ -564,7 +564,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

@ -25,6 +25,8 @@ import org.apache.lucene.document.FieldSelector;
import org.apache.lucene.document.FieldSelectorResult;
import org.apache.lucene.document.Fieldable;
import org.apache.lucene.document.NumericField;
import org.apache.lucene.index.IOContext.Context;
import org.apache.lucene.index.MergePolicy.OneMerge;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.BufferedIndexInput;
import org.apache.lucene.store.Directory;
@ -81,7 +83,7 @@ public final class FieldsReader implements Cloneable {
/** 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();
@ -110,18 +112,18 @@ public final class FieldsReader implements Cloneable {
}
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

@ -65,14 +65,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

@ -0,0 +1,73 @@
package org.apache.lucene.index;
/**
* 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 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(Context context) {
this(context, null);
}
private IOContext(boolean readOnce) {
this.context = Context.READ;
this.mergeInfo = null;
this.readOnce = readOnce;
}
public IOContext (MergeInfo mergeInfo) {
this(Context.MERGE, mergeInfo);
}
private IOContext (Context context, MergeInfo mergeInfo ) {
assert context != Context.MERGE || mergeInfo != null;
this.context = context;
this.readOnce = false;
this.mergeInfo = mergeInfo;
}
}

View File

@ -21,6 +21,7 @@ import org.apache.lucene.document.Document;
import org.apache.lucene.document.FieldSelector;
import org.apache.lucene.search.FieldCache; // javadocs
import org.apache.lucene.search.Similarity;
import org.apache.lucene.index.IOContext.Context;
import org.apache.lucene.index.codecs.Codec;
import org.apache.lucene.index.codecs.CodecProvider;
import org.apache.lucene.index.codecs.PerDocValues;
@ -1433,13 +1434,14 @@ public abstract class IndexReader implements Cloneable,Closeable {
Directory dir = null;
CompoundFileReader 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 = new CompoundFileReader(dir, filename);
cfr = new CompoundFileReader(dir, filename, context);
String [] files = cfr.listAll();
ArrayUtil.mergeSort(files); // sort the array of filename so that the output is more readable
@ -1449,7 +1451,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

@ -39,7 +39,9 @@ import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.index.DocumentsWriterPerThread.FlushedSegment;
import org.apache.lucene.index.FieldInfos.FieldNumberBiMap;
import org.apache.lucene.index.IOContext.Context;
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
import org.apache.lucene.index.MergePolicy.OneMerge;
import org.apache.lucene.index.PayloadProcessorProvider.DirPayloadProcessor;
import org.apache.lucene.index.SegmentCodecs.SegmentCodecsBuilder;
import org.apache.lucene.index.codecs.CodecProvider;
@ -592,8 +594,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 {
@ -609,8 +611,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());
}
/**
@ -624,18 +626,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;
// }
// nocommit 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) {
@ -2183,6 +2187,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
SegmentInfo newSegment = flushedSegment.segmentInfo;
setDiagnostics(newSegment, "flush");
IOContext context = new IOContext(Context.FLUSH);
boolean success = false;
try {
@ -2190,7 +2196,7 @@ 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
CompoundFileWriter cfsWriter = new CompoundFileWriter(directory, compoundFileName);
CompoundFileWriter cfsWriter = new CompoundFileWriter(directory, compoundFileName, context);
for(String fileName : newSegment.files()) {
cfsWriter.addFile(fileName);
}
@ -2222,7 +2228,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.deletedDocuments.write(directory, delFileName);
flushedSegment.deletedDocuments.write(directory, delFileName, context);
success2 = true;
} finally {
if (!success2) {
@ -2391,11 +2397,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);
@ -2454,7 +2462,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
for (IndexReader reader : readers) // add new indexes
merger.add(reader);
// nocommit - we should pass a MergeInfo here into merge to create corresponding IOContext instances?
int docCount = merger.merge(); // merge 'em
final FieldInfos fieldInfos = merger.fieldInfos();
@ -2475,7 +2483,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
// Now create the compound file if needed
if (useCompoundFile) {
merger.createCompoundFile(mergedName + ".cfs", info);
IOContext context = new IOContext(new MergeInfo(info.docCount, info.sizeInBytes(true), true, false));
merger.createCompoundFile(mergedName + ".cfs", info, context);
// delete new non cfs files directly: they were never
// registered with IFD
@ -2499,10 +2508,10 @@ 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();
CompoundFileWriter cfsWriter = new CompoundFileWriter(directory, segFileName);
CompoundFileWriter cfsWriter = new CompoundFileWriter(directory, segFileName, context);
for (String file : files) {
String newFileName = segName + IndexFileNames.stripSegmentName(file);
if (!IndexFileNames.matchesExtension(file, IndexFileNames.DELETES_EXTENSION)
@ -2510,7 +2519,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
cfsWriter.addFile(file, info.dir);
} else {
assert !directory.fileExists(newFileName): "file \"" + newFileName + "\" already exists";
info.dir.copy(directory, file, newFileName);
info.dir.copy(directory, file, newFileName, context);
}
}
@ -2524,7 +2533,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,
@ -2560,7 +2569,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());
@ -3416,6 +3425,8 @@ 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());
@ -3439,7 +3450,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);
@ -3493,7 +3504,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) {
@ -3563,7 +3574,9 @@ 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);
// nocommit should we use another flag "isMergedSegment" or a "READ" context here?
final SegmentReader mergedReader = readerPool.get(merge.info, loadDocStores, context, termsIndexDivisor);
try {
if (poolReaders && mergedSegmentWarmer != null) {
mergedSegmentWarmer.warm(mergedReader);

View File

@ -0,0 +1,35 @@
package org.apache.lucene.index;
/**
* 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.
*/
//nocommit javadoc
public class MergeInfo {
public int totalDocCount;
public long estimatedMergeBytes; // used by IndexWriter
boolean isExternal; // used by IndexWriter
boolean optimize; // used by IndexWriter
public MergeInfo(int totalDocCount, long estimatedMergeBytes, boolean isExternal, boolean optimize) {
this.totalDocCount = totalDocCount;
this.estimatedMergeBytes = estimatedMergeBytes;
this.isExternal = isExternal;
this.optimize = optimize;
}
}

View File

@ -189,6 +189,10 @@ public abstract class MergePolicy implements java.io.Closeable {
}
return total;
}
public MergeInfo getMergeInfo() {
return new MergeInfo(totalDocCount, estimatedMergeBytes, isExternal, optimize);
}
}
/**

View File

@ -21,6 +21,7 @@ import java.io.IOException;
import java.util.Collection;
import java.util.Map;
import org.apache.lucene.index.IOContext.Context;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.IOUtils;
@ -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

@ -99,7 +99,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 +111,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 +187,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 +212,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 +229,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

@ -45,7 +45,7 @@ final class SegmentCoreReaders {
final Directory dir;
final Directory cfsDir;
final int readBufferSize;
final IOContext context;
final int termsIndexDivisor;
private final SegmentReader owner;
@ -57,7 +57,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)");
@ -65,7 +65,7 @@ final class SegmentCoreReaders {
segment = si.name;
final SegmentCodecs segmentCodecs = si.getSegmentCodecs();
this.readBufferSize = readBufferSize;
this.context = context;
this.dir = dir;
boolean success = false;
@ -73,7 +73,7 @@ final class SegmentCoreReaders {
try {
Directory dir0 = dir;
if (si.getUseCompoundFile()) {
cfsReader = new CompoundFileReader(dir, IndexFileNames.segmentFileName(segment, "", IndexFileNames.COMPOUND_FILE_EXTENSION), readBufferSize);
cfsReader = new CompoundFileReader(dir, IndexFileNames.segmentFileName(segment, "", IndexFileNames.COMPOUND_FILE_EXTENSION), context);
dir0 = cfsReader;
}
cfsDir = dir0;
@ -82,7 +82,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;
@ -163,7 +163,7 @@ final class SegmentCoreReaders {
assert storeCFSReader == null;
storeCFSReader = new CompoundFileReader(dir,
IndexFileNames.segmentFileName(si.getDocStoreSegment(), "", IndexFileNames.COMPOUND_FILE_STORE_EXTENSION),
readBufferSize);
context);
storeDir = storeCFSReader;
assert storeDir != null;
} else {
@ -175,7 +175,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 = new CompoundFileReader(dir, IndexFileNames.segmentFileName(segment, "", IndexFileNames.COMPOUND_FILE_EXTENSION), readBufferSize);
cfsReader = new CompoundFileReader(dir, IndexFileNames.segmentFileName(segment, "", IndexFileNames.COMPOUND_FILE_EXTENSION), context);
}
storeDir = cfsReader;
assert storeDir != null;
@ -185,7 +185,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:
@ -194,7 +194,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

@ -245,7 +245,7 @@ public final class SegmentInfo implements Cloneable {
}
final Directory dirToTest;
if (isCompoundFile) {
dirToTest = new CompoundFileReader(dir, IndexFileNames.segmentFileName(storesSegment, "", ext));
dirToTest = new CompoundFileReader(dir, IndexFileNames.segmentFileName(storesSegment, "", ext), IOContext.READONCE);
} else {
dirToTest = dir;
}
@ -264,7 +264,7 @@ public final class SegmentInfo implements Cloneable {
Directory dir0 = dir;
if (isCompoundFile && checkCompoundFile) {
dir0 = new CompoundFileReader(dir, IndexFileNames.segmentFileName(name,
"", IndexFileNames.COMPOUND_FILE_EXTENSION));
"", IndexFileNames.COMPOUND_FILE_EXTENSION), IOContext.READONCE);
}
try {
fieldInfos = new FieldInfos(dir0, IndexFileNames.segmentFileName(name,

View File

@ -32,6 +32,7 @@ import java.util.Map;
import java.util.Set;
import org.apache.lucene.index.FieldInfos.FieldNumberBiMap;
import org.apache.lucene.index.IOContext.Context;
import org.apache.lucene.index.codecs.CodecProvider;
import org.apache.lucene.index.codecs.DefaultSegmentInfosWriter;
import org.apache.lucene.index.codecs.SegmentInfosReader;
@ -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

@ -24,6 +24,7 @@ import java.util.Collection;
import java.util.List;
import org.apache.lucene.document.Document;
import org.apache.lucene.index.IOContext.Context;
import org.apache.lucene.index.IndexReader.FieldOption;
import org.apache.lucene.index.MergePolicy.MergeAbortedException;
import org.apache.lucene.index.codecs.Codec;
@ -127,12 +128,12 @@ 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();
CompoundFileWriter cfsWriter = new CompoundFileWriter(directory, fileName, checkAbort);
CompoundFileWriter cfsWriter = new CompoundFileWriter(directory, fileName, context, checkAbort);
for (String file : files) {
assert !IndexFileNames.matchesExtension(file, IndexFileNames.DELETES_EXTENSION)
: ".del file is not allowed in .cfs: " + file;
@ -232,8 +233,8 @@ final class SegmentMerger {
int docCount = 0;
setMatchingSegmentReaders();
final FieldsWriter fieldsWriter = new FieldsWriter(directory, segment);
// nocommit - should we rather use IOContext.MERGE here?
final FieldsWriter fieldsWriter = new FieldsWriter(directory, segment, IOContext.DEFAULT);
try {
int idx = 0;
@ -268,8 +269,8 @@ 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);
//nocommit if Merge then what to initialize OneMerge with ?
segmentWriteState = new SegmentWriteState(null, directory, segment, fieldInfos, docCount, termIndexInterval, codecInfo, null, IOContext.DEFAULT);
return docCount;
}
@ -354,8 +355,9 @@ final class SegmentMerger {
* @throws IOException
*/
private final void mergeVectors() throws IOException {
//nocommit Putting MERGE context here would lead to assert error. What should MergeInfo be initialized with here?
TermVectorsWriter termVectorsWriter =
new TermVectorsWriter(directory, segment, fieldInfos);
new TermVectorsWriter(directory, segment, fieldInfos, new IOContext(Context.DEFAULT));
try {
int idx = 0;
@ -614,7 +616,8 @@ final class SegmentMerger {
for (FieldInfo fi : fieldInfos) {
if (fi.isIndexed && !fi.omitNorms) {
if (output == null) {
output = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.NORMS_EXTENSION));
//nocommit Putting MERGE context here would lead to assert error. What should MergeInfo be initialized with here?
output = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.NORMS_EXTENSION), new IOContext(Context.DEFAULT));
output.writeBytes(SegmentNorms.NORMS_HEADER, SegmentNorms.NORMS_HEADER.length);
}
for (IndexReader reader : readers) {

View File

@ -20,6 +20,8 @@ package org.apache.lucene.index;
import java.io.IOException;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.lucene.index.IOContext.Context;
import org.apache.lucene.index.MergePolicy.OneMerge;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
@ -219,7 +221,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, IOContext.DEFAULT);
boolean success = false;
try {
try {

View File

@ -26,7 +26,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 +37,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

@ -30,7 +30,6 @@ import java.util.concurrent.atomic.AtomicInteger;
import org.apache.lucene.document.Document;
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.IndexInput;
import org.apache.lucene.util.BitVector;
@ -46,7 +45,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 +86,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 +98,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.loadDeletedDocs();
instance.openNorms(instance.core.cfsDir, readBufferSize);
instance.openNorms(instance.core.cfsDir, context);
success = true;
} finally {
@ -161,7 +158,7 @@ public class SegmentReader extends IndexReader implements Cloneable {
private void loadDeletedDocs() throws IOException {
// NOTE: the bitvector is stored using the regular directory, not cfs
if (hasDeletions(si)) {
deletedDocs = new BitVector(directory(), si.getDelFileName());
deletedDocs = new BitVector(directory(), si.getDelFileName(), IOContext.DEFAULT);
deletedDocsRef = new AtomicInteger(1);
assert checkDeletedCounts();
if (deletedDocs.size() != si.docCount) {
@ -250,7 +247,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;
@ -295,7 +291,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 {
@ -337,7 +333,7 @@ public class SegmentReader extends IndexReader implements Cloneable {
final String delFileName = si.getDelFileName();
boolean success = false;
try {
deletedDocs.write(directory(), delFileName);
deletedDocs.write(directory(), delFileName, IOContext.DEFAULT);
success = true;
} finally {
if (!success) {
@ -576,7 +572,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) {
@ -600,7 +596,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();
@ -610,7 +606,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

@ -51,9 +51,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 +65,7 @@ public class SegmentWriteState {
this.termIndexInterval = termIndexInterval;
this.segmentCodecs = segmentCodecs;
codecId = -1;
this.context = context;
}
/**
@ -76,6 +79,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.index.IOContext.Context;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.RamUsageEstimator;
@ -77,7 +78,8 @@ final class StoredFieldsWriter {
private synchronized void initFieldsWriter() throws IOException {
if (fieldsWriter == null) {
fieldsWriter = new FieldsWriter(docWriter.directory, docWriter.getSegment());
// nocommit - is this always a flush here or should we have a IOContext argument?
fieldsWriter = new FieldsWriter(docWriter.directory, docWriter.getSegment(), new IOContext(Context.FLUSH));
lastDocID = 0;
}
}

View File

@ -17,6 +17,8 @@ package org.apache.lucene.index;
* limitations under the License.
*/
import org.apache.lucene.index.IOContext.Context;
import org.apache.lucene.index.MergePolicy.OneMerge;
import org.apache.lucene.store.BufferedIndexInput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IndexInput;
@ -61,29 +63,24 @@ class TermVectorsReader implements Cloneable {
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,6 +20,7 @@ package org.apache.lucene.index;
import java.io.IOException;
import java.util.Map;
import org.apache.lucene.index.IOContext.Context;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
@ -89,13 +90,14 @@ final class TermVectorsTermsWriter extends TermsHashConsumer {
if (tvx == null) {
boolean success = false;
try {
IOContext context = new IOContext(Context.FLUSH);
// 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

@ -31,15 +31,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

@ -29,6 +29,7 @@ import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.FieldsEnum;
import org.apache.lucene.index.IOContext;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.TermState;
@ -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

@ -22,6 +22,7 @@ import java.util.TreeMap;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IOContext;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.values.Bytes;
import org.apache.lucene.index.values.IndexDocValues;
@ -57,6 +58,7 @@ public class DefaultDocValuesProducer extends PerDocValues {
*/
public DefaultDocValuesProducer(SegmentInfo si, Directory dir,
FieldInfos fieldInfo, int codecId) throws IOException {
//nocommit this needs an IOContext
docValues = load(fieldInfo, si.name, si.docCount, dir, codecId);
}
@ -120,6 +122,7 @@ public class DefaultDocValuesProducer extends PerDocValues {
*/
protected IndexDocValues loadDocValues(int docCount, Directory dir, String id,
ValueType type) throws IOException {
// nocommit this needs an IOContext too
switch (type) {
case INTS:
return Ints.getValues(dir, id, false);

View File

@ -22,11 +22,13 @@ import java.io.IOException;
import org.apache.lucene.index.CompoundFileReader;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.FieldsReader;
import org.apache.lucene.index.IOContext;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexFormatTooOldException;
import org.apache.lucene.index.IndexFormatTooNewException;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentInfos;
import org.apache.lucene.index.IOContext.Context;
import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IndexInput;
@ -39,10 +41,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);
@ -70,11 +72,11 @@ public class DefaultSegmentInfosReader extends SegmentInfosReader {
if (si.getDocStoreIsCompoundFile()) {
dir = new CompoundFileReader(dir, IndexFileNames.segmentFileName(
si.getDocStoreSegment(), "",
IndexFileNames.COMPOUND_FILE_STORE_EXTENSION), 1024);
IndexFileNames.COMPOUND_FILE_STORE_EXTENSION), context);
}
} else if (si.getUseCompoundFile()) {
dir = new CompoundFileReader(dir, IndexFileNames.segmentFileName(
si.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION), 1024);
si.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION), context);
}
try {
@ -108,8 +110,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

@ -19,8 +19,10 @@ package org.apache.lucene.index.codecs;
import java.io.IOException;
import org.apache.lucene.index.IOContext;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentInfos;
import org.apache.lucene.index.IOContext.Context;
import org.apache.lucene.store.ChecksumIndexOutput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IndexOutput;
@ -54,9 +56,10 @@ 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);
//nocommit should this context always be flush?
IndexOutput out = createOutput(dir, segmentFileName, context);
boolean success = false;
try {
out.writeInt(FORMAT_CURRENT); // write FORMAT
@ -77,9 +80,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

@ -21,6 +21,8 @@ import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IOContext;
import org.apache.lucene.index.IOContext.Context;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CodecUtil;
@ -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

@ -19,6 +19,7 @@ package org.apache.lucene.index.codecs;
import java.io.IOException;
import org.apache.lucene.index.IOContext;
import org.apache.lucene.index.SegmentInfos;
import org.apache.lucene.store.Directory;
@ -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

@ -19,6 +19,7 @@ package org.apache.lucene.index.codecs;
import java.io.IOException;
import org.apache.lucene.index.IOContext;
import org.apache.lucene.index.SegmentInfos;
import org.apache.lucene.store.Directory;
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

@ -27,6 +27,8 @@ import java.util.Iterator;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IOContext;
import org.apache.lucene.index.IOContext.Context;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.store.Directory;
@ -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

@ -242,7 +242,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
@ -697,8 +697,9 @@ public class MemoryCodec extends Codec {
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
//nocommit its seems due to the nature of this codec that we should use IOContext.READONCE here where applicable.
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, state.context);
final SortedMap<String,TermsReader> fields = new TreeMap<String,TermsReader>();

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

@ -31,6 +31,7 @@ import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.FieldsEnum;
import org.apache.lucene.index.IOContext;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.Term;
@ -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 CompoundFileReader)) {
dir0 = cfsReader = new CompoundFileReader(dir, IndexFileNames.segmentFileName(si.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION), readBufferSize);
dir0 = cfsReader = new CompoundFileReader(dir, 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

@ -21,6 +21,7 @@ import java.io.IOException;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IOContext;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.store.Directory;
@ -94,7 +95,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;
@ -108,7 +109,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;
@ -116,7 +117,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

@ -110,7 +110,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
@ -122,7 +122,7 @@ public class PulsingCodec extends Codec {
state.fieldInfos,
state.segmentInfo.name,
state.termsIndexDivisor,
state.codecId);
state.codecId, state.context);
success = true;
} finally {
if (!success) {
@ -136,7 +136,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;

View File

@ -17,6 +17,7 @@ package org.apache.lucene.index.codecs.sep;
* limitations under the License.
*/
import org.apache.lucene.index.IOContext;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.BufferedIndexInput;
@ -24,10 +25,10 @@ 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 IntIndexInput openInput(Directory dir, String fileName, IOContext context) throws IOException;
public abstract IntIndexOutput createOutput(Directory dir, String fileName) throws IOException;
}

View File

@ -23,6 +23,7 @@ import java.util.Collection;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IOContext;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.TermState;
@ -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

@ -132,11 +132,11 @@ public final class SepPostingsWriterImpl extends PostingsWriterBase {
// 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

@ -58,7 +58,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

@ -89,7 +89,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;
@ -98,7 +98,7 @@ public class StandardCodec extends Codec {
state.fieldInfos,
state.segmentInfo.name,
state.termsIndexDivisor,
state.codecId);
state.codecId, state.context);
success = true;
} finally {
if (!success) {
@ -113,7 +113,7 @@ public class StandardCodec extends Codec {
state.fieldInfos,
state.segmentInfo.name,
postings,
state.readBufferSize,
state.context,
TERMS_CACHE_SIZE,
state.codecId);
success = true;

View File

@ -23,6 +23,7 @@ import java.util.Collection;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IOContext;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.TermState;
@ -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

@ -23,6 +23,7 @@ import java.util.Collection;
import java.util.Comparator;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.lucene.index.IOContext;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.values.IndexDocValues.SortedSource;
import org.apache.lucene.index.values.IndexDocValues.Source;
@ -108,6 +109,8 @@ public final class Bytes {
public static Writer getWriter(Directory dir, String id, Mode mode,
Comparator<BytesRef> comp, boolean fixedSize, AtomicLong bytesUsed)
throws IOException {
//nocommit this and all the blow need an IOContext too
// TODO -- i shouldn't have to specify fixed? can
// track itself & do the write thing at write time?
if (comp == null) {
@ -158,6 +161,8 @@ public final class Bytes {
*/
public static IndexDocValues getValues(Directory dir, String id, Mode mode,
boolean fixedSize, int maxDoc) throws IOException {
//nocommit this and all the readers below need an IOContext too
// TODO -- I can peek @ header to determing fixed/mode?
if (fixedSize) {
if (mode == Mode.STRAIGHT) {
@ -344,18 +349,18 @@ public final class Bytes {
protected BytesWriterBase(Directory dir, String id, String codecName,
int version, boolean initIndex, ByteBlockPool pool,
AtomicLong bytesUsed) throws IOException {
AtomicLong bytesUsed, IOContext context) throws IOException {
super(bytesUsed);
this.id = id;
this.pool = pool;
datOut = dir.createOutput(IndexFileNames.segmentFileName(id, "",
DATA_EXTENSION));
DATA_EXTENSION), context);
boolean success = false;
try {
CodecUtil.writeHeader(datOut, codecName, version);
if (initIndex) {
idxOut = dir.createOutput(IndexFileNames.segmentFileName(id, "",
INDEX_EXTENSION));
INDEX_EXTENSION), context);
CodecUtil.writeHeader(idxOut, codecName, version);
} else {
idxOut = null;
@ -428,16 +433,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

@ -20,6 +20,7 @@ package org.apache.lucene.index.values;
import java.io.IOException;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.lucene.index.IOContext;
import org.apache.lucene.index.values.Bytes.BytesBaseSource;
import org.apache.lucene.index.values.Bytes.BytesReaderBase;
import org.apache.lucene.index.values.Bytes.BytesWriterBase;
@ -56,6 +57,7 @@ class FixedDerefBytesImpl {
BytesRefHash.DEFAULT_CAPACITY, bytesUsed));
public Writer(Directory dir, String id, AtomicLong bytesUsed)
throws IOException {
//nocommit this needs an IOContext too
this(dir, id, new DirectTrackingAllocator(ByteBlockPool.BYTE_BLOCK_SIZE, bytesUsed),
bytesUsed);
}
@ -63,7 +65,7 @@ class FixedDerefBytesImpl {
public Writer(Directory dir, String id, Allocator allocator,
AtomicLong bytesUsed) throws IOException {
super(dir, id, CODEC_NAME, VERSION_CURRENT, true,
new ByteBlockPool(allocator), bytesUsed);
new ByteBlockPool(allocator), bytesUsed, IOContext.DEFAULT);
docToID = new int[1];
bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT); // TODO BytesRefHash
// uses bytes too!
@ -134,7 +136,7 @@ class FixedDerefBytesImpl {
private final int size;
Reader(Directory dir, String id, int maxDoc) throws IOException {
super(dir, id, CODEC_NAME, VERSION_START, true);
super(dir, id, CODEC_NAME, VERSION_START, true, IOContext.DEFAULT);
size = datIn.readInt();
}

View File

@ -21,6 +21,7 @@ import java.io.IOException;
import java.util.Comparator;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.lucene.index.IOContext;
import org.apache.lucene.index.values.Bytes.BytesBaseSortedSource;
import org.apache.lucene.index.values.Bytes.BytesReaderBase;
import org.apache.lucene.index.values.Bytes.BytesWriterBase;
@ -63,6 +64,7 @@ class FixedSortedBytesImpl {
public Writer(Directory dir, String id, Comparator<BytesRef> comp,
AtomicLong bytesUsed) throws IOException {
//nocommit this needs an IOContext too
this(dir, id, comp, new DirectTrackingAllocator(ByteBlockPool.BYTE_BLOCK_SIZE, bytesUsed),
bytesUsed);
}
@ -70,7 +72,7 @@ class FixedSortedBytesImpl {
public Writer(Directory dir, String id, Comparator<BytesRef> comp,
Allocator allocator, AtomicLong bytesUsed) throws IOException {
super(dir, id, CODEC_NAME, VERSION_CURRENT, true,
new ByteBlockPool(allocator), bytesUsed);
new ByteBlockPool(allocator), bytesUsed, IOContext.DEFAULT);
docToEntry = new int[1];
// docToEntry[0] = -1;
bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT);
@ -161,7 +163,7 @@ class FixedSortedBytesImpl {
private final int size;
public Reader(Directory dir, String id, int maxDoc) throws IOException {
super(dir, id, CODEC_NAME, VERSION_START, true);
super(dir, id, CODEC_NAME, VERSION_START, true, IOContext.DEFAULT);
size = datIn.readInt();
}

View File

@ -19,6 +19,7 @@ package org.apache.lucene.index.values;
import java.io.IOException;
import org.apache.lucene.index.IOContext;
import org.apache.lucene.index.values.Bytes.BytesBaseSource;
import org.apache.lucene.index.values.Bytes.BytesReaderBase;
import org.apache.lucene.index.values.Bytes.BytesWriterBase;
@ -47,7 +48,8 @@ class FixedStraightBytesImpl {
private byte[] oneRecord;
public Writer(Directory dir, String id) throws IOException {
super(dir, id, CODEC_NAME, VERSION_CURRENT, false, null, null);
//nocommit this needs an IOContext too
super(dir, id, CODEC_NAME, VERSION_CURRENT, false, null, null, IOContext.DEFAULT);
}
@ -126,7 +128,7 @@ class FixedStraightBytesImpl {
private final int maxDoc;
Reader(Directory dir, String id, int maxDoc) throws IOException {
super(dir, id, CODEC_NAME, VERSION_START, false);
super(dir, id, CODEC_NAME, VERSION_START, false, IOContext.DEFAULT);
size = datIn.readInt();
this.maxDoc = maxDoc;
}

View File

@ -20,6 +20,8 @@ import java.io.IOException;
import java.util.Collection;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.lucene.index.IOContext;
import org.apache.lucene.index.IOContext.Context;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.values.IndexDocValues.Source;
import org.apache.lucene.store.Directory;
@ -65,7 +67,8 @@ public class Floats {
public static IndexDocValues getValues(Directory dir, String id, int maxDoc)
throws IOException {
return new FloatsReader(dir, id, maxDoc);
//nocommit this needs an IOContext too
return new FloatsReader(dir, id, maxDoc, IOContext.READ);
}
abstract static class FloatsWriter extends Writer {
@ -76,12 +79,12 @@ public class Floats {
private final byte precision;
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;
datOut = dir.createOutput(IndexFileNames.segmentFileName(id, "",
Writer.DATA_EXTENSION));
Writer.DATA_EXTENSION), context);
boolean success = false;
try {
CodecUtil.writeHeader(datOut, CODEC_NAME, VERSION_CURRENT);
@ -146,7 +149,7 @@ public class Floats {
protected Float4Writer(Directory dir, String id, AtomicLong bytesUsed)
throws IOException {
super(dir, id, 4, bytesUsed);
super(dir, id, 4, bytesUsed, new IOContext(Context.FLUSH));
}
@Override
@ -189,7 +192,7 @@ public class Floats {
protected Float8Writer(Directory dir, String id, AtomicLong bytesUsed)
throws IOException {
super(dir, id, 8, bytesUsed);
super(dir, id, 8, bytesUsed, new IOContext(Context.FLUSH));
}
@Override
@ -237,10 +240,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

@ -20,6 +20,7 @@ package org.apache.lucene.index.values;
import java.io.IOException;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.lucene.index.IOContext;
import org.apache.lucene.index.values.IntsImpl.IntsReader;
import org.apache.lucene.index.values.IntsImpl.IntsWriter;
import org.apache.lucene.store.Directory;
@ -36,11 +37,12 @@ public class Ints {
public static Writer getWriter(Directory dir, String id,
boolean useFixedArray, AtomicLong bytesUsed) throws IOException {
// TODO - implement fixed?!
return new IntsWriter(dir, id, bytesUsed);
return new IntsWriter(dir, id, bytesUsed, IOContext.DEFAULT);
}
public static IndexDocValues getValues(Directory dir, String id,
boolean useFixedArray) throws IOException {
return new IntsReader(dir, id);
//nocommit this needs an IOContext too
return new IntsReader(dir, id, IOContext.DEFAULT);
}
}

View File

@ -20,6 +20,7 @@ import java.io.IOException;
import java.util.Collection;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.lucene.index.IOContext;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IndexInput;
@ -58,11 +59,11 @@ class IntsImpl {
private int lastDocId = -1;
private IndexOutput datOut;
protected IntsWriter(Directory dir, String id, AtomicLong bytesUsed)
protected IntsWriter(Directory dir, String id, AtomicLong bytesUsed, IOContext context)
throws IOException {
super(bytesUsed);
datOut = dir.createOutput(IndexFileNames.segmentFileName(id, "",
DATA_EXTENSION));
DATA_EXTENSION), context);
boolean success = false;
try {
CodecUtil.writeHeader(datOut, CODEC_NAME, VERSION_CURRENT);
@ -185,9 +186,9 @@ class IntsImpl {
private final IndexInput datIn;
private final boolean packed;
protected IntsReader(Directory dir, String id) throws IOException {
protected IntsReader(Directory dir, String id, IOContext context) throws IOException {
datIn = dir.openInput(IndexFileNames.segmentFileName(id, "",
Writer.DATA_EXTENSION));
Writer.DATA_EXTENSION), context);
boolean success = false;
try {
CodecUtil.checkHeader(datIn, CODEC_NAME, VERSION_START, VERSION_START);

View File

@ -20,6 +20,7 @@ package org.apache.lucene.index.values;
import java.io.IOException;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.lucene.index.IOContext;
import org.apache.lucene.index.values.Bytes.BytesBaseSource;
import org.apache.lucene.index.values.Bytes.BytesReaderBase;
import org.apache.lucene.index.values.Bytes.BytesWriterBase;
@ -117,6 +118,7 @@ class VarDerefBytesImpl {
public Writer(Directory dir, String id, AtomicLong bytesUsed)
throws IOException {
//nocommit this needs an IOContext too
this(dir, id, new DirectTrackingAllocator(ByteBlockPool.BYTE_BLOCK_SIZE, bytesUsed),
bytesUsed);
}
@ -124,7 +126,7 @@ class VarDerefBytesImpl {
public Writer(Directory dir, String id, Allocator allocator,
AtomicLong bytesUsed) throws IOException {
super(dir, id, CODEC_NAME, VERSION_CURRENT, true,
new ByteBlockPool(allocator), bytesUsed);
new ByteBlockPool(allocator), bytesUsed, IOContext.DEFAULT);
docToAddress = new int[1];
bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT);
}
@ -202,7 +204,7 @@ 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);
super(dir, id, CODEC_NAME, VERSION_START, true, IOContext.DEFAULT);
}
@Override

View File

@ -22,6 +22,7 @@ import java.util.Arrays;
import java.util.Comparator;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.lucene.index.IOContext;
import org.apache.lucene.index.values.Bytes.BytesBaseSortedSource;
import org.apache.lucene.index.values.Bytes.BytesReaderBase;
import org.apache.lucene.index.values.Bytes.BytesWriterBase;
@ -62,6 +63,7 @@ class VarSortedBytesImpl {
public Writer(Directory dir, String id, Comparator<BytesRef> comp,
AtomicLong bytesUsed) throws IOException {
//nocommit this needs an IOContext too
this(dir, id, comp, new DirectTrackingAllocator(ByteBlockPool.BYTE_BLOCK_SIZE, bytesUsed),
bytesUsed);
}
@ -69,7 +71,7 @@ class VarSortedBytesImpl {
public Writer(Directory dir, String id, Comparator<BytesRef> comp,
Allocator allocator, AtomicLong bytesUsed) throws IOException {
super(dir, id, CODEC_NAME, VERSION_CURRENT, true,
new ByteBlockPool(allocator), bytesUsed);
new ByteBlockPool(allocator), bytesUsed, IOContext.DEFAULT);
this.comp = comp;
docToEntry = new int[1];
docToEntry[0] = -1;
@ -157,7 +159,7 @@ class VarSortedBytesImpl {
public static class Reader extends BytesReaderBase {
Reader(Directory dir, String id, int maxDoc) throws IOException {
super(dir, id, CODEC_NAME, VERSION_START, true);
super(dir, id, CODEC_NAME, VERSION_START, true, IOContext.DEFAULT);
}
@Override

View File

@ -20,6 +20,7 @@ package org.apache.lucene.index.values;
import java.io.IOException;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.lucene.index.IOContext;
import org.apache.lucene.index.values.Bytes.BytesBaseSource;
import org.apache.lucene.index.values.Bytes.BytesReaderBase;
import org.apache.lucene.index.values.Bytes.BytesWriterBase;
@ -51,7 +52,8 @@ class VarStraightBytesImpl {
public Writer(Directory dir, String id, AtomicLong bytesUsed)
throws IOException {
super(dir, id, CODEC_NAME, VERSION_CURRENT, true, null, bytesUsed);
//nocommit this needs an IOContext too
super(dir, id, CODEC_NAME, VERSION_CURRENT, true, null, bytesUsed, IOContext.DEFAULT);
docToAddress = new long[1];
bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT);
}
@ -122,7 +124,7 @@ class VarStraightBytesImpl {
private final int maxDoc;
Reader(Directory dir, String id, int maxDoc) throws IOException {
super(dir, id, CODEC_NAME, VERSION_START, true);
super(dir, id, CODEC_NAME, VERSION_START, true, IOContext.DEFAULT);
this.maxDoc = maxDoc;
}

View File

@ -43,8 +43,9 @@ public abstract class BufferedIndexInput extends IndexInput {
public BufferedIndexInput() {}
/** Inits BufferedIndexInput with a specific bufferSize */
//nocommit To cleanup class variable bufferSize as the the default size is always used
public BufferedIndexInput(int bufferSize) {
checkBufferSize(bufferSize);
checkBufferSize(BufferedIndexInput.BUFFER_SIZE);
this.bufferSize = bufferSize;
}

View File

@ -22,6 +22,7 @@ import java.io.IOException;
import java.io.Closeable;
import java.util.Collection; // for javadocs
import org.apache.lucene.index.IOContext;
import org.apache.lucene.util.IOUtils;
/** A Directory is a flat list of files. Files may be written once, when they
@ -87,7 +88,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 +104,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 +111,7 @@ public abstract class Directory implements Closeable {
* parameter are {@link FSDirectory} and {@link
* org.apache.lucene.index.CompoundFileReader}.
*/
public IndexInput openInput(String name, int bufferSize) throws IOException {
return openInput(name);
}
public abstract IndexInput openInput(String name, IOContext context) throws IOException;
/** Construct a {@link Lock}.
* @param name the name of the lock file
@ -199,9 +194,9 @@ 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 {
IndexOutput os = to.createOutput(dest);
IndexInput is = openInput(src);
public void copy(Directory to, String src, String dest, IOContext context) throws IOException {
IndexOutput os = to.createOutput(dest, context);
IndexInput is = openInput(src, context);
IOException priorException = null;
try {
is.copyBytes(os, is.length());

View File

@ -29,6 +29,7 @@ import java.util.HashSet;
import java.util.Set;
import java.util.concurrent.Future;
import org.apache.lucene.index.IOContext;
import org.apache.lucene.util.ThreadInterruptedException;
import org.apache.lucene.util.Constants;
@ -286,7 +287,7 @@ 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);
@ -319,13 +320,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();

View File

@ -25,6 +25,8 @@ import java.util.List;
import java.util.Set;
import java.util.HashSet;
import org.apache.lucene.index.IOContext;
/**
* Expert: A Directory instance that switches files between
* two other Directory instances.
@ -125,8 +127,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,7 +147,7 @@ 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);
}
}

View File

@ -31,6 +31,7 @@ import java.security.PrivilegedExceptionAction;
import java.security.PrivilegedActionException;
import java.lang.reflect.Method;
import org.apache.lucene.index.IOContext;
import org.apache.lucene.util.Constants;
/** File-based {@link Directory} implementation that uses
@ -208,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");
@ -399,4 +400,5 @@ public class MMapDirectory extends FSDirectory {
}
}
}
}

View File

@ -23,6 +23,7 @@ import java.nio.ByteBuffer;
import java.nio.channels.ClosedChannelException; // javadoc @link
import java.nio.channels.FileChannel;
import java.util.concurrent.Future; // javadoc
import org.apache.lucene.index.IOContext;
/**
* An {@link FSDirectory} implementation that uses java.nio's FileChannel's
@ -73,9 +74,9 @@ 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());
}
protected static class NIOFSIndexInput extends SimpleFSDirectory.SimpleFSIndexInput {
@ -87,8 +88,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();
}
@ -178,4 +179,5 @@ public class NIOFSDirectory extends FSDirectory {
}
}
}
}

View File

@ -27,6 +27,8 @@ import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.lucene.index.IOContext;
/**
* A memory-resident {@link Directory} implementation. Locking
* implementation is by default the {@link SingleInstanceLockFactory}
@ -38,7 +40,7 @@ public class RAMDirectory extends Directory {
// *****
// Lock acquisition sequence: RAMDirectory, then RAMFile
// *****
// *****
/** Constructs an empty {@link Directory}. */
public RAMDirectory() {
@ -65,14 +67,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 +151,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 +178,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

@ -21,6 +21,8 @@ import java.io.File;
import java.io.IOException;
import java.io.RandomAccessFile;
import org.apache.lucene.index.IOContext;
/** A straightforward implementation of {@link FSDirectory}
* using java.io.RandomAccessFile. However, this class has
* poor concurrent performance (multiple threads will
@ -51,9 +53,9 @@ 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());
}
protected static class SimpleFSIndexInput extends BufferedIndexInput {
@ -85,8 +87,9 @@ public class SimpleFSDirectory extends FSDirectory {
// LUCENE-1566 - maximum read length on a 32bit JVM to prevent incorrect OOM
protected final int chunkSize;
public SimpleFSIndexInput(File path, int bufferSize, int chunkSize) throws IOException {
super(bufferSize);
public SimpleFSIndexInput(File path, IOContext context, int chunkSize) throws IOException {
//nocommit Use IOContext to decide bufferSize instead of BufferedIndexInput.BUFFER_SIZE
super(BufferedIndexInput.BUFFER_SIZE);
file = new Descriptor(path, "r");
this.chunkSize = chunkSize;
}

View File

@ -19,6 +19,9 @@ package org.apache.lucene.util;
import java.io.IOException;
import org.apache.lucene.index.IOContext;
import org.apache.lucene.index.IOContext.Context;
import org.apache.lucene.index.MergePolicy.OneMerge;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
@ -162,8 +165,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 {
if (isSparse()) {
writeDgaps(output); // sparse bit-set more efficiently saved as d-gaps.
@ -220,8 +223,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 {
size = input.readInt(); // read size
if (size == -1) {

View File

@ -20,6 +20,7 @@ package org.apache.lucene.index.codecs.mockintblock;
import java.io.IOException;
import java.util.Set;
import org.apache.lucene.index.IOContext;
import org.apache.lucene.index.PerDocWriteState;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentWriteState;
@ -84,8 +85,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 {
@ -103,7 +104,7 @@ public class MockFixedIntBlockCodec extends Codec {
@Override
public IntIndexOutput createOutput(Directory dir, String fileName) throws IOException {
IndexOutput out = dir.createOutput(fileName);
IndexOutput out = dir.createOutput(fileName, IOContext.DEFAULT);
boolean success = false;
try {
FixedIntBlockIndexOutput ret = new FixedIntBlockIndexOutput(out, blockSize) {
@ -160,7 +161,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 +171,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 +187,7 @@ public class MockFixedIntBlockCodec extends Codec {
state.fieldInfos,
state.segmentInfo.name,
postingsReader,
state.readBufferSize,
state.context,
StandardCodec.TERMS_CACHE_SIZE,
state.codecId);
success = true;

View File

@ -20,6 +20,7 @@ package org.apache.lucene.index.codecs.mockintblock;
import java.io.IOException;
import java.util.Set;
import org.apache.lucene.index.IOContext;
import org.apache.lucene.index.PerDocWriteState;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentWriteState;
@ -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) {
@ -107,7 +108,7 @@ public class MockVariableIntBlockCodec extends Codec {
@Override
public IntIndexOutput createOutput(Directory dir, String fileName) throws IOException {
final IndexOutput out = dir.createOutput(fileName);
final IndexOutput out = dir.createOutput(fileName, IOContext.DEFAULT);
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;

View File

@ -25,6 +25,7 @@ import java.util.Random;
import java.util.Set;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IOContext;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.PerDocWriteState;
import org.apache.lucene.index.SegmentInfo;
@ -103,14 +104,14 @@ 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
@ -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;

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;

View File

@ -18,6 +18,7 @@ package org.apache.lucene.index.codecs.mocksep;
*/
import org.apache.lucene.store.Directory;
import org.apache.lucene.index.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,8 +28,8 @@ 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 {

View File

@ -19,6 +19,7 @@ package org.apache.lucene.index.codecs.mocksep;
import java.io.IOException;
import org.apache.lucene.index.IOContext;
import org.apache.lucene.index.codecs.sep.IntIndexInput;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.Directory;
@ -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

@ -21,6 +21,7 @@ import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.CodecUtil;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.index.IOContext;
import org.apache.lucene.index.codecs.sep.IntIndexOutput;
import java.io.IOException;
@ -36,7 +37,8 @@ public class MockSingleIntIndexOutput extends IntIndexOutput {
final static int VERSION_CURRENT = VERSION_START;
public MockSingleIntIndexOutput(Directory dir, String fileName) throws IOException {
out = dir.createOutput(fileName);
//nocommit pass IOContext in via ctor!
out = dir.createOutput(fileName, IOContext.DEFAULT);
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

@ -51,7 +51,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

@ -22,6 +22,7 @@ import java.io.Closeable;
import java.io.IOException;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IOContext;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.codecs.preflex.PreFlexCodec;
import org.apache.lucene.index.codecs.preflex.TermInfo;
@ -123,9 +124,10 @@ final class TermInfosWriter implements Closeable {
indexInterval = interval;
fieldInfos = fis;
isIndex = isi;
// nocommit pass IOContext in via ctor
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

View File

@ -31,6 +31,7 @@ import java.util.Map;
import java.util.Random;
import java.util.Set;
import org.apache.lucene.index.IOContext;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.codecs.CodecProvider;
import org.apache.lucene.util.LuceneTestCase;
@ -196,7 +197,8 @@ public class MockDirectoryWrapper extends Directory {
long length = fileLength(name);
byte[] zeroes = new byte[256];
long upto = 0;
IndexOutput out = delegate.createOutput(name);
//nocommit - randomize the IOContext here?
IndexOutput out = delegate.createOutput(name, IOContext.DEFAULT);
while(upto < length) {
final int limit = (int) Math.min(length-upto, zeroes.length);
out.writeBytes(zeroes, 0, limit);
@ -205,7 +207,8 @@ public class MockDirectoryWrapper extends Directory {
out.close();
} else if (count % 3 == 2) {
// Truncate the file:
IndexOutput out = delegate.createOutput(name);
//nocommit - randomize the IOContext here?
IndexOutput out = delegate.createOutput(name, IOContext.DEFAULT);
out.setLength(fileLength(name)/2);
out.close();
}
@ -337,7 +340,7 @@ public class MockDirectoryWrapper extends Directory {
}
@Override
public synchronized IndexOutput createOutput(String name) throws IOException {
public synchronized IndexOutput createOutput(String name, IOContext context) throws IOException {
maybeYield();
if (crashed)
throw new IOException("cannot createOutput after crash");
@ -372,7 +375,8 @@ public class MockDirectoryWrapper extends Directory {
}
//System.out.println(Thread.currentThread().getName() + ": MDW: create " + name);
IndexOutput io = new MockIndexOutputWrapper(this, delegate.createOutput(name), name);
// nocommit - randomize the IOContext here?
IndexOutput io = new MockIndexOutputWrapper(this, delegate.createOutput(name, context), name);
addFileHandle(io, name, false);
openFilesForWrite.add(name);
@ -401,7 +405,7 @@ public class MockDirectoryWrapper extends Directory {
}
@Override
public synchronized IndexInput openInput(String name) throws IOException {
public synchronized IndexInput openInput(String name, IOContext context) throws IOException {
maybeYield();
if (!delegate.fileExists(name))
throw new FileNotFoundException(name);
@ -412,7 +416,8 @@ public class MockDirectoryWrapper extends Directory {
throw fillOpenTrace(new IOException("MockDirectoryWrapper: file \"" + name + "\" is still open for writing"), name, false);
}
IndexInput ii = new MockIndexInputWrapper(this, name, delegate.openInput(name));
// nocommit - randomize IOContext here?
IndexInput ii = new MockIndexInputWrapper(this, name, delegate.openInput(name, context));
addFileHandle(ii, name, true);
return ii;
}
@ -637,9 +642,10 @@ public class MockDirectoryWrapper extends Directory {
}
@Override
public synchronized void copy(Directory to, String src, String dest) throws IOException {
public synchronized void copy(Directory to, String src, String dest, IOContext context) throws IOException {
maybeYield();
delegate.copy(to, src, dest);
// randomize the IOContext here?
delegate.copy(to, src, dest, context);
}
}

View File

@ -1056,7 +1056,8 @@ public abstract class LuceneTestCase extends Assert {
public static MockDirectoryWrapper newDirectory(Random r, Directory d) throws IOException {
Directory impl = newDirectoryImpl(r, TEST_DIRECTORY);
for (String file : d.listAll()) {
d.copy(impl, file, file);
//nocommit randomiz the IOContext here?
d.copy(impl, file, file, IOContext.DEFAULT);
}
MockDirectoryWrapper dir = new MockDirectoryWrapper(r, impl);
stores.put(dir, Thread.currentThread().getStackTrace());

View File

@ -395,7 +395,7 @@ public class TestAddIndexes extends LuceneTestCase {
setMergePolicy(newLogMergePolicy(4))
);
writer.addIndexes(aux, new MockDirectoryWrapper(random, new RAMDirectory(aux)));
writer.addIndexes(aux, new MockDirectoryWrapper(random, new RAMDirectory(aux, IOContext.DEFAULT)));
assertEquals(1060, writer.maxDoc());
assertEquals(1000, writer.getDocCount(0));
writer.close();
@ -430,7 +430,7 @@ public class TestAddIndexes extends LuceneTestCase {
setMergePolicy(newLogMergePolicy(4))
);
writer.addIndexes(aux, new MockDirectoryWrapper(random, new RAMDirectory(aux)));
writer.addIndexes(aux, new MockDirectoryWrapper(random, new RAMDirectory(aux, IOContext.DEFAULT)));
assertEquals(1020, writer.maxDoc());
assertEquals(1000, writer.getDocCount(0));
writer.close();
@ -665,7 +665,7 @@ public class TestAddIndexes extends LuceneTestCase {
final Directory[] dirs = new Directory[NUM_COPY];
for(int k=0;k<NUM_COPY;k++)
dirs[k] = new MockDirectoryWrapper(random, new RAMDirectory(dir));
dirs[k] = new MockDirectoryWrapper(random, new RAMDirectory(dir, IOContext.DEFAULT));
int j=0;

View File

@ -31,6 +31,7 @@ import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.Fieldable;
import org.apache.lucene.document.NumericField;
import org.apache.lucene.index.IOContext.Context;
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
import org.apache.lucene.search.DefaultSimilarity;
import org.apache.lucene.search.DocIdSetIterator;
@ -536,7 +537,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
// figure out which field number corresponds to
// "content", and then set our expected file names below
// accordingly:
CompoundFileReader cfsReader = new CompoundFileReader(dir, "_0.cfs");
CompoundFileReader cfsReader = new CompoundFileReader(dir, "_0.cfs", IOContext.READ);
FieldInfos fieldInfos = new FieldInfos(cfsReader, "_0.fnm");
int contentFieldIndex = -1;
for (FieldInfo fi : fieldInfos) {

View File

@ -25,6 +25,7 @@ import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.Field.Store;
import org.apache.lucene.index.IOContext.Context;
import org.apache.lucene.index.codecs.CodecProvider;
import org.apache.lucene.index.codecs.FieldsConsumer;
import org.apache.lucene.index.codecs.FieldsProducer;
@ -242,7 +243,7 @@ public class TestCodecs extends LuceneTestCase {
this.write(fieldInfos, dir, fields, true);
final SegmentInfo si = new SegmentInfo(SEGMENT, 10000, dir, false, clonedFieldInfos.buildSegmentCodecs(false), clonedFieldInfos);
final FieldsProducer reader = si.getSegmentCodecs().codec().fieldsProducer(new SegmentReadState(dir, si, fieldInfos, 64, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR));
final FieldsProducer reader = si.getSegmentCodecs().codec().fieldsProducer(new SegmentReadState(dir, si, fieldInfos, IOContext.READ, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR));
final FieldsEnum fieldsEnum = reader.iterator();
assertNotNull(fieldsEnum.next());
@ -297,7 +298,7 @@ public class TestCodecs extends LuceneTestCase {
if (VERBOSE) {
System.out.println("TEST: now read postings");
}
final FieldsProducer terms = si.getSegmentCodecs().codec().fieldsProducer(new SegmentReadState(dir, si, fieldInfos, 1024, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR));
final FieldsProducer terms = si.getSegmentCodecs().codec().fieldsProducer(new SegmentReadState(dir, si, fieldInfos, IOContext.READ, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR));
final Verify[] threads = new Verify[NUM_TEST_THREADS-1];
for(int i=0;i<NUM_TEST_THREADS-1;i++) {
@ -591,7 +592,7 @@ public class TestCodecs extends LuceneTestCase {
final int termIndexInterval = _TestUtil.nextInt(random, 13, 27);
final SegmentCodecs codecInfo = fieldInfos.buildSegmentCodecs(false);
final SegmentWriteState state = new SegmentWriteState(null, dir, SEGMENT, fieldInfos, 10000, termIndexInterval, codecInfo, null);
final SegmentWriteState state = new SegmentWriteState(null, dir, SEGMENT, fieldInfos, 10000, termIndexInterval, codecInfo, null, new IOContext(Context.FLUSH));
final FieldsConsumer consumer = state.segmentCodecs.codec().fieldsConsumer(state);
Arrays.sort(fields);

View File

@ -23,6 +23,8 @@ import java.io.File;
import org.apache.lucene.util.LuceneTestCase;
import junit.framework.TestSuite;
import junit.textui.TestRunner;
import org.apache.lucene.index.IOContext.Context;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IndexInput;
@ -72,7 +74,7 @@ public class TestCompoundFile extends LuceneTestCase
private void createRandomFile(Directory dir, String name, int size)
throws IOException
{
IndexOutput os = dir.createOutput(name);
IndexOutput os = dir.createOutput(name, new IOContext(Context.FLUSH));
for (int i=0; i<size; i++) {
byte b = (byte) (Math.random() * 256);
os.writeByte(b);
@ -90,7 +92,7 @@ public class TestCompoundFile extends LuceneTestCase
int size)
throws IOException
{
IndexOutput os = dir.createOutput(name);
IndexOutput os = dir.createOutput(name, new IOContext(Context.FLUSH));
for (int i=0; i < size; i++) {
os.writeByte(start);
start ++;
@ -200,13 +202,13 @@ public class TestCompoundFile extends LuceneTestCase
for (int i=0; i<data.length; i++) {
String name = "t" + data[i];
createSequenceFile(dir, name, (byte) 0, data[i]);
CompoundFileWriter csw = new CompoundFileWriter(dir, name + ".cfs");
CompoundFileWriter csw = new CompoundFileWriter(dir, name + ".cfs", IOContext.DEFAULT);
csw.addFile(name);
csw.close();
CompoundFileReader csr = new CompoundFileReader(dir, name + ".cfs");
IndexInput expected = dir.openInput(name);
IndexInput actual = csr.openInput(name);
CompoundFileReader csr = new CompoundFileReader(dir, name + ".cfs", IOContext.DEFAULT);
IndexInput expected = dir.openInput(name, IOContext.DEFAULT);
IndexInput actual = csr.openInput(name, IOContext.DEFAULT);
assertSameStreams(name, expected, actual);
assertSameSeekBehavior(name, expected, actual);
expected.close();
@ -223,21 +225,21 @@ public class TestCompoundFile extends LuceneTestCase
createSequenceFile(dir, "d1", (byte) 0, 15);
createSequenceFile(dir, "d2", (byte) 0, 114);
CompoundFileWriter csw = new CompoundFileWriter(dir, "d.csf");
CompoundFileWriter csw = new CompoundFileWriter(dir, "d.csf", IOContext.DEFAULT);
csw.addFile("d1");
csw.addFile("d2");
csw.close();
CompoundFileReader csr = new CompoundFileReader(dir, "d.csf");
IndexInput expected = dir.openInput("d1");
IndexInput actual = csr.openInput("d1");
CompoundFileReader csr = new CompoundFileReader(dir, "d.csf", IOContext.DEFAULT);
IndexInput expected = dir.openInput("d1", IOContext.DEFAULT);
IndexInput actual = csr.openInput("d1", IOContext.DEFAULT);
assertSameStreams("d1", expected, actual);
assertSameSeekBehavior("d1", expected, actual);
expected.close();
actual.close();
expected = dir.openInput("d2");
actual = csr.openInput("d2");
expected = dir.openInput("d2", IOContext.DEFAULT);
actual = csr.openInput("d2", IOContext.DEFAULT);
assertSameStreams("d2", expected, actual);
assertSameSeekBehavior("d2", expected, actual);
expected.close();
@ -273,7 +275,7 @@ public class TestCompoundFile extends LuceneTestCase
createRandomFile(dir, segment + ".notIn2", 51);
// Now test
CompoundFileWriter csw = new CompoundFileWriter(dir, "test.cfs");
CompoundFileWriter csw = new CompoundFileWriter(dir, "test.cfs", IOContext.DEFAULT);
final String data[] = new String[] {
".zero", ".one", ".ten", ".hundred", ".big1", ".big2", ".big3",
".big4", ".big5", ".big6", ".big7"
@ -283,10 +285,10 @@ public class TestCompoundFile extends LuceneTestCase
}
csw.close();
CompoundFileReader csr = new CompoundFileReader(dir, "test.cfs");
CompoundFileReader csr = new CompoundFileReader(dir, "test.cfs", IOContext.DEFAULT);
for (int i=0; i<data.length; i++) {
IndexInput check = dir.openInput(segment + data[i]);
IndexInput test = csr.openInput(segment + data[i]);
IndexInput check = dir.openInput(segment + data[i], IOContext.DEFAULT);
IndexInput test = csr.openInput(segment + data[i], IOContext.DEFAULT);
assertSameStreams(data[i], check, test);
assertSameSeekBehavior(data[i], check, test);
test.close();
@ -302,7 +304,7 @@ public class TestCompoundFile extends LuceneTestCase
* the size of each file is 1000 bytes.
*/
private void setUp_2() throws IOException {
CompoundFileWriter cw = new CompoundFileWriter(dir, "f.comp");
CompoundFileWriter cw = new CompoundFileWriter(dir, "f.comp", IOContext.DEFAULT);
for (int i=0; i<20; i++) {
createSequenceFile(dir, "f" + i, (byte) 0, 2000);
cw.addFile("f" + i);
@ -319,13 +321,13 @@ public class TestCompoundFile extends LuceneTestCase
throws IOException
{
// Setup the test file - we need more than 1024 bytes
IndexOutput os = fsdir.createOutput(file);
IndexOutput os = fsdir.createOutput(file, IOContext.DEFAULT);
for(int i=0; i<2000; i++) {
os.writeByte((byte) i);
}
os.close();
IndexInput in = fsdir.openInput(file);
IndexInput in = fsdir.openInput(file, IOContext.DEFAULT);
// This read primes the buffer in IndexInput
in.readByte();
@ -369,16 +371,16 @@ public class TestCompoundFile extends LuceneTestCase
public void testClonedStreamsClosing() throws IOException {
setUp_2();
CompoundFileReader cr = new CompoundFileReader(dir, "f.comp");
CompoundFileReader cr = new CompoundFileReader(dir, "f.comp", IOContext.DEFAULT);
// basic clone
IndexInput expected = dir.openInput("f11");
IndexInput expected = dir.openInput("f11", IOContext.DEFAULT);
// this test only works for FSIndexInput
assertTrue(_TestHelper.isSimpleFSIndexInput(expected));
assertTrue(_TestHelper.isSimpleFSIndexInputOpen(expected));
IndexInput one = cr.openInput("f11");
IndexInput one = cr.openInput("f11", IOContext.DEFAULT);
assertTrue(isCSIndexInputOpen(one));
IndexInput two = (IndexInput) one.clone();
@ -426,14 +428,14 @@ public class TestCompoundFile extends LuceneTestCase
*/
public void testRandomAccess() throws IOException {
setUp_2();
CompoundFileReader cr = new CompoundFileReader(dir, "f.comp");
CompoundFileReader cr = new CompoundFileReader(dir, "f.comp", IOContext.DEFAULT);
// Open two files
IndexInput e1 = dir.openInput("f11");
IndexInput e2 = dir.openInput("f3");
IndexInput e1 = dir.openInput("f11", IOContext.DEFAULT);
IndexInput e2 = dir.openInput("f3", IOContext.DEFAULT);
IndexInput a1 = cr.openInput("f11");
IndexInput a2 = dir.openInput("f3");
IndexInput a1 = cr.openInput("f11", IOContext.DEFAULT);
IndexInput a2 = dir.openInput("f3", IOContext.DEFAULT);
// Seek the first pair
e1.seek(100);
@ -505,11 +507,11 @@ public class TestCompoundFile extends LuceneTestCase
*/
public void testRandomAccessClones() throws IOException {
setUp_2();
CompoundFileReader cr = new CompoundFileReader(dir, "f.comp");
CompoundFileReader cr = new CompoundFileReader(dir, "f.comp", IOContext.DEFAULT);
// Open two files
IndexInput e1 = cr.openInput("f11");
IndexInput e2 = cr.openInput("f3");
IndexInput e1 = cr.openInput("f11", IOContext.DEFAULT);
IndexInput e2 = cr.openInput("f3", IOContext.DEFAULT);
IndexInput a1 = (IndexInput) e1.clone();
IndexInput a2 = (IndexInput) e2.clone();
@ -582,11 +584,11 @@ public class TestCompoundFile extends LuceneTestCase
public void testFileNotFound() throws IOException {
setUp_2();
CompoundFileReader cr = new CompoundFileReader(dir, "f.comp");
CompoundFileReader cr = new CompoundFileReader(dir, "f.comp", IOContext.DEFAULT);
// Open two files
try {
cr.openInput("bogus");
cr.openInput("bogus", IOContext.DEFAULT);
fail("File not found");
} catch (IOException e) {
@ -600,8 +602,8 @@ public class TestCompoundFile extends LuceneTestCase
public void testReadPastEOF() throws IOException {
setUp_2();
CompoundFileReader cr = new CompoundFileReader(dir, "f.comp");
IndexInput is = cr.openInput("f2");
CompoundFileReader cr = new CompoundFileReader(dir, "f.comp", IOContext.DEFAULT);
IndexInput is = cr.openInput("f2", IOContext.DEFAULT);
is.seek(is.length() - 10);
byte b[] = new byte[100];
is.readBytes(b, 0, 10);
@ -631,7 +633,7 @@ public class TestCompoundFile extends LuceneTestCase
* will correctly increment the file pointer.
*/
public void testLargeWrites() throws IOException {
IndexOutput os = dir.createOutput("testBufferStart.txt");
IndexOutput os = dir.createOutput("testBufferStart.txt", IOContext.DEFAULT);
byte[] largeBuf = new byte[2048];
for (int i=0; i<largeBuf.length; i++) {
@ -653,13 +655,13 @@ public class TestCompoundFile extends LuceneTestCase
createSequenceFile(dir, "d1", (byte) 0, 15);
Directory newDir = newDirectory();
CompoundFileWriter csw = new CompoundFileWriter(newDir, "d.csf");
CompoundFileWriter csw = new CompoundFileWriter(newDir, "d.csf", IOContext.DEFAULT);
csw.addFile("d1", dir);
csw.close();
CompoundFileReader csr = new CompoundFileReader(newDir, "d.csf");
IndexInput expected = dir.openInput("d1");
IndexInput actual = csr.openInput("d1");
CompoundFileReader csr = new CompoundFileReader(newDir, "d.csf", IOContext.DEFAULT);
IndexInput expected = dir.openInput("d1", IOContext.DEFAULT);
IndexInput actual = csr.openInput("d1", IOContext.DEFAULT);
assertSameStreams("d1", expected, actual);
assertSameSeekBehavior("d1", expected, actual);
expected.close();

View File

@ -32,7 +32,9 @@ import junit.textui.TestRunner;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.IOContext.Context;
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
import org.apache.lucene.index.MergePolicy.OneMerge;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.LuceneTestCase;
@ -193,8 +195,9 @@ public class TestDoc extends LuceneTestCase {
private SegmentInfo merge(SegmentInfo si1, SegmentInfo si2, String merged, boolean useCompoundFile)
throws Exception {
SegmentReader r1 = SegmentReader.get(true, si1, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR);
SegmentReader r2 = SegmentReader.get(true, si2, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR);
IOContext context = IOContext.READ;
SegmentReader r1 = SegmentReader.get(true, si1, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, context);
SegmentReader r2 = SegmentReader.get(true, si2, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, context);
SegmentMerger merger = new SegmentMerger(si1.dir, IndexWriterConfig.DEFAULT_TERM_INDEX_INTERVAL, merged, null, null, new FieldInfos());
@ -208,7 +211,7 @@ public class TestDoc extends LuceneTestCase {
false, merger.getSegmentCodecs(), fieldInfos);
if (useCompoundFile) {
Collection<String> filesToDelete = merger.createCompoundFile(merged + ".cfs", info);
Collection<String> filesToDelete = merger.createCompoundFile(merged + ".cfs", info, IOContext.DEFAULT);
info.setUseCompoundFile(true);
for (final String fileToDelete : filesToDelete)
si1.dir.deleteFile(fileToDelete);
@ -220,7 +223,7 @@ public class TestDoc extends LuceneTestCase {
private void printSegment(PrintWriter out, SegmentInfo si)
throws Exception {
SegmentReader reader = SegmentReader.get(true, si, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR);
SegmentReader reader = SegmentReader.get(true, si, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, IOContext.READ);
for (int i = 0; i < reader.numDocs(); i++)
out.println(reader.document(i));

View File

@ -148,7 +148,7 @@ public class TestDocTermOrds extends LuceneTestCase {
@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;
@ -158,7 +158,7 @@ public class TestDocTermOrds extends LuceneTestCase {
state.segmentInfo.name,
state.termsIndexDivisor,
BytesRef.getUTF8SortedAsUnicodeComparator(),
state.codecId);
state.codecId, state.context);
success = true;
} finally {
if (!success) {
@ -173,7 +173,7 @@ public class TestDocTermOrds extends LuceneTestCase {
state.fieldInfos,
state.segmentInfo.name,
postings,
state.readBufferSize,
state.context,
TERMS_CACHE_SIZE,
state.codecId);
success = true;

View File

@ -34,6 +34,7 @@ import org.apache.lucene.document.Field.Index;
import org.apache.lucene.document.Field.Store;
import org.apache.lucene.document.Field.TermVector;
import org.apache.lucene.document.Fieldable;
import org.apache.lucene.index.IOContext.Context;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.AttributeSource;
import org.apache.lucene.util.BytesRef;
@ -68,7 +69,7 @@ public class TestDocumentWriter extends LuceneTestCase {
SegmentInfo info = writer.newestSegment();
writer.close();
//After adding the document, we should be able to read it back in
SegmentReader reader = SegmentReader.get(true, info, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR);
SegmentReader reader = SegmentReader.get(true, info, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, IOContext.READ);
assertTrue(reader != null);
Document doc = reader.document(0);
assertTrue(doc != null);
@ -129,7 +130,7 @@ public class TestDocumentWriter extends LuceneTestCase {
writer.commit();
SegmentInfo info = writer.newestSegment();
writer.close();
SegmentReader reader = SegmentReader.get(true, info, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR);
SegmentReader reader = SegmentReader.get(true, info, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, IOContext.READ);
DocsAndPositionsEnum termPositions = MultiFields.getTermPositionsEnum(reader, MultiFields.getDeletedDocs(reader),
"repeated", new BytesRef("repeated"));
@ -193,7 +194,7 @@ public class TestDocumentWriter extends LuceneTestCase {
writer.commit();
SegmentInfo info = writer.newestSegment();
writer.close();
SegmentReader reader = SegmentReader.get(true, info, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR);
SegmentReader reader = SegmentReader.get(true, info, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, IOContext.READ);
DocsAndPositionsEnum termPositions = reader.fields().terms("f1").docsAndPositions(reader.getDeletedDocs(), new BytesRef("a"), null);
assertTrue(termPositions.nextDoc() != termPositions.NO_MORE_DOCS);
@ -237,7 +238,7 @@ public class TestDocumentWriter extends LuceneTestCase {
writer.commit();
SegmentInfo info = writer.newestSegment();
writer.close();
SegmentReader reader = SegmentReader.get(true, info, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR);
SegmentReader reader = SegmentReader.get(true, info, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, IOContext.READ);
DocsAndPositionsEnum termPositions = reader.fields().terms("preanalyzed").docsAndPositions(reader.getDeletedDocs(), new BytesRef("term1"), null);
assertTrue(termPositions.nextDoc() != termPositions.NO_MORE_DOCS);

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