LUCENE-2793: next iteration on IOContext

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/LUCENE2793@1141592 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Simon Willnauer 2011-06-30 16:03:28 +00:00
parent e8355de9e6
commit e9937378c9
125 changed files with 529 additions and 420 deletions

View File

@ -99,13 +99,12 @@ public class AppendingCodec extends Codec {
boolean success = false;
try {
//nocommit we should pass the IOContext down to the TermIndexReader
indexReader = new AppendingTermsIndexReader(state.dir,
state.fieldInfos,
state.segmentInfo.name,
state.termsIndexDivisor,
BytesRef.getUTF8SortedAsUnicodeComparator(),
state.codecId);
state.codecId, state.context);
success = true;
} finally {
if (!success) {
@ -155,6 +154,6 @@ public class AppendingCodec extends Codec {
@Override
public PerDocValues docsProducer(SegmentReadState state) throws IOException {
return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId);
return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId, state.context);
}
}

View File

@ -20,9 +20,9 @@ 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.IOContext;
import org.apache.lucene.store.IndexInput;
public class AppendingSegmentInfosReader extends DefaultSegmentInfosReader {

View File

@ -19,9 +19,9 @@ 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.IOContext;
import org.apache.lucene.store.IndexOutput;
public class AppendingSegmentInfosWriter extends DefaultSegmentInfosWriter {

View File

@ -20,12 +20,12 @@ 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;
import org.apache.lucene.index.codecs.TermsIndexReaderBase;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.CodecUtil;

View File

@ -21,9 +21,9 @@ 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.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CodecUtil;
@ -31,9 +31,9 @@ import org.apache.lucene.util.CodecUtil;
public class AppendingTermsIndexReader extends FixedGapTermsIndexReader {
public AppendingTermsIndexReader(Directory dir, FieldInfos fieldInfos,
String segment, int indexDivisor, Comparator<BytesRef> termComp, int codecId)
String segment, int indexDivisor, Comparator<BytesRef> termComp, int codecId, IOContext context)
throws IOException {
super(dir, fieldInfos, segment, indexDivisor, termComp, codecId, IOContext.DEFAULT);
super(dir, fieldInfos, segment, indexDivisor, termComp, codecId, context);
}
@Override

View File

@ -25,7 +25,6 @@ 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

View File

@ -24,10 +24,8 @@ 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

View File

@ -20,7 +20,6 @@ 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

View File

@ -27,7 +27,6 @@ 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;
@ -41,6 +40,7 @@ import org.apache.lucene.index.codecs.CodecProvider;
import org.apache.lucene.index.codecs.SegmentInfosReader;
import org.apache.lucene.index.codecs.SegmentInfosWriter;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.MockDirectoryWrapper;
import org.apache.lucene.store.RAMDirectory;

View File

@ -24,7 +24,6 @@ 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;

View File

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

View File

@ -22,11 +22,10 @@ import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.store.FSDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import 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;

View File

@ -18,6 +18,7 @@ package org.apache.lucene.index;
*/
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.BufferedIndexInput;
import org.apache.lucene.store.IndexOutput;

View File

@ -23,6 +23,7 @@ import java.util.LinkedList;
import org.apache.lucene.index.codecs.MergeState;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.IOUtils;

View File

@ -32,9 +32,9 @@ import java.util.concurrent.ConcurrentHashMap;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.FieldSelector;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.Lock;
import org.apache.lucene.store.LockObtainFailedException;
import org.apache.lucene.index.IOContext.Context;
import org.apache.lucene.index.codecs.CodecProvider;
import org.apache.lucene.index.codecs.PerDocValues;
import org.apache.lucene.util.Bits;

View File

@ -28,9 +28,11 @@ 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.store.FlushInfo;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IOContext.Context;
import org.apache.lucene.util.BitVector;
import org.apache.lucene.util.ByteBlockPool.Allocator;
import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator;
@ -429,7 +431,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, new IOContext(Context.FLUSH));
fieldInfos.buildSegmentCodecs(true), pendingDeletes, new IOContext(new FlushInfo(numDocsInRAM, bytesUsed())));
final double startMBUsed = parent.flushControl.netBytes() / 1024. / 1024.;
// Apply delete-by-docID now (delete-byDocID only
// happens when an exception is hit processing that

View File

@ -33,6 +33,7 @@ import org.apache.lucene.index.SegmentCodecs.SegmentCodecsBuilder;
import org.apache.lucene.index.codecs.CodecProvider;
import org.apache.lucene.index.values.ValueType;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.CodecUtil;

View File

@ -25,11 +25,10 @@ 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;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.CloseableThreadLocal;

View File

@ -23,6 +23,7 @@ import org.apache.lucene.document.Document;
import org.apache.lucene.document.Fieldable;
import org.apache.lucene.document.NumericField;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.IOUtils;

View File

@ -21,7 +21,6 @@ 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;

View File

@ -39,9 +39,7 @@ 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;
@ -49,8 +47,12 @@ import org.apache.lucene.search.Query;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.BufferedIndexInput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FlushInfo;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.Lock;
import org.apache.lucene.store.LockObtainFailedException;
import org.apache.lucene.store.MergeInfo;
import org.apache.lucene.store.IOContext.Context;
import org.apache.lucene.util.BitVector;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.Constants;
@ -2188,7 +2190,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
setDiagnostics(newSegment, "flush");
IOContext context = new IOContext(Context.FLUSH);
IOContext context = new IOContext(new FlushInfo(newSegment.docCount, newSegment.sizeInBytes(true)));
boolean success = false;
try {
@ -2447,6 +2449,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
*/
public void addIndexes(IndexReader... readers) throws CorruptIndexException, IOException {
ensureOpen();
int numDocs = 0;
try {
if (infoStream != null)
@ -2454,11 +2457,16 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
flush(false, true);
String mergedName = newSegmentName();
for (IndexReader indexReader : readers) {
numDocs += indexReader.numDocs();
}
final IOContext context = new IOContext(new MergeInfo(numDocs, -1, true, false));
// TODO: somehow we should fix this merge so it's
// abortable so that IW.close(false) is able to stop it
SegmentMerger merger = new SegmentMerger(directory, config.getTermIndexInterval(),
mergedName, null, payloadProcessorProvider,
globalFieldNumberMap.newFieldInfos(SegmentCodecsBuilder.create(codecs)));
globalFieldNumberMap.newFieldInfos(SegmentCodecsBuilder.create(codecs)), context);
for (IndexReader reader : readers) // add new indexes
merger.add(reader);
@ -2483,7 +2491,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
// Now create the compound file if needed
if (useCompoundFile) {
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
@ -3429,7 +3436,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
IOContext context = new IOContext(merge.getMergeInfo());
SegmentMerger merger = new SegmentMerger(directory, config.getTermIndexInterval(), mergedName, merge,
payloadProcessorProvider, merge.info.getFieldInfos());
payloadProcessorProvider, merge.info.getFieldInfos(), context);
if (infoStream != null) {
message("merging " + merge.segString(directory) + " mergeVectors=" + merge.info.getFieldInfos().hasVectors());

View File

@ -18,6 +18,7 @@ package org.apache.lucene.index;
*/
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.MergeInfo;
import org.apache.lucene.util.SetOnce;
import org.apache.lucene.util.SetOnce.AlreadySetException;

View File

@ -21,8 +21,8 @@ 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.store.IOContext.Context;
import org.apache.lucene.util.IOUtils;
// TODO FI: norms could actually be stored as doc store

View File

@ -36,6 +36,7 @@ import org.apache.lucene.index.codecs.TermsConsumer;
import org.apache.lucene.index.codecs.DocValuesConsumer;
import org.apache.lucene.index.values.IndexDocValues;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.IOUtils;
/**

View File

@ -24,6 +24,7 @@ import org.apache.lucene.index.codecs.Codec;
import org.apache.lucene.index.codecs.FieldsProducer;
import org.apache.lucene.index.codecs.PerDocValues;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
/** Holds core readers that are shared (unchanged) when
* SegmentReader is cloned or reopened */

View File

@ -31,6 +31,7 @@ import org.apache.lucene.index.codecs.Codec;
import org.apache.lucene.index.codecs.CodecProvider;
import org.apache.lucene.index.codecs.DefaultSegmentInfosWriter;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.Constants;

View File

@ -32,12 +32,12 @@ 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;
import org.apache.lucene.index.codecs.SegmentInfosWriter;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.NoSuchDirectoryException;

View File

@ -24,7 +24,6 @@ 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;
@ -33,6 +32,7 @@ import org.apache.lucene.index.codecs.MergeState;
import org.apache.lucene.index.codecs.PerDocConsumer;
import org.apache.lucene.index.codecs.PerDocValues;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.Bits;
@ -68,8 +68,10 @@ final class SegmentMerger {
private SegmentWriteState segmentWriteState;
private PayloadProcessorProvider payloadProcessorProvider;
private IOContext context;
SegmentMerger(Directory dir, int termIndexInterval, String name, MergePolicy.OneMerge merge, PayloadProcessorProvider payloadProcessorProvider, FieldInfos fieldInfos) {
SegmentMerger(Directory dir, int termIndexInterval, String name, MergePolicy.OneMerge merge, PayloadProcessorProvider payloadProcessorProvider, FieldInfos fieldInfos, IOContext context) {
this.payloadProcessorProvider = payloadProcessorProvider;
directory = dir;
segment = name;
@ -85,6 +87,7 @@ final class SegmentMerger {
};
}
this.termIndexInterval = termIndexInterval;
this.context = context;
}
public FieldInfos fieldInfos() {
@ -234,8 +237,7 @@ final class SegmentMerger {
setMatchingSegmentReaders();
// nocommit - should we rather use IOContext.MERGE here?
final FieldsWriter fieldsWriter = new FieldsWriter(directory, segment, IOContext.DEFAULT);
final FieldsWriter fieldsWriter = new FieldsWriter(directory, segment, context);
try {
int idx = 0;
for (IndexReader reader : readers) {
@ -269,8 +271,7 @@ final class SegmentMerger {
// entering the index. See LUCENE-1282 for
// details.
throw new RuntimeException("mergeFields produced an invalid result: docCount is " + docCount + " but fdx file size is " + fdxFileLength + " file=" + fileName + " file exists?=" + directory.fileExists(fileName) + "; now aborting this merge to prevent index corruption");
//nocommit if Merge then what to initialize OneMerge with ?
segmentWriteState = new SegmentWriteState(null, directory, segment, fieldInfos, docCount, termIndexInterval, codecInfo, null, IOContext.DEFAULT);
segmentWriteState = new SegmentWriteState(null, directory, segment, fieldInfos, docCount, termIndexInterval, codecInfo, null, context);
return docCount;
}
@ -355,9 +356,8 @@ 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 IOContext(Context.DEFAULT));
new TermVectorsWriter(directory, segment, fieldInfos, context);
try {
int idx = 0;
@ -616,8 +616,7 @@ final class SegmentMerger {
for (FieldInfo fi : fieldInfos) {
if (fi.isIndexed && !fi.omitNorms) {
if (output == null) {
//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 = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.NORMS_EXTENSION), context);
output.writeBytes(SegmentNorms.NORMS_HEADER, SegmentNorms.NORMS_HEADER.length);
}
for (IndexReader reader : readers) {

View File

@ -20,11 +20,10 @@ 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.FlushInfo;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
/**
* Byte[] referencing is used because a new norm object needs
* to be created for each clone, and the byte array is all
@ -221,7 +220,8 @@ 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, IOContext.DEFAULT);
//nocommit not sure if this is the correct information provided to FlushInfo
IndexOutput out = owner.directory().createOutput(normFileName, new IOContext(new FlushInfo(si.docCount, 0)));
boolean success = false;
try {
try {

View File

@ -18,6 +18,7 @@ package org.apache.lucene.index;
*/
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
/**
* @lucene.experimental

View File

@ -31,6 +31,7 @@ import org.apache.lucene.document.Document;
import org.apache.lucene.document.FieldSelector;
import org.apache.lucene.index.codecs.PerDocValues;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.BitVector;
import org.apache.lucene.util.Bits;

View File

@ -20,6 +20,7 @@ package org.apache.lucene.index;
import java.io.PrintStream;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.BitVector;
/**

View File

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

View File

@ -17,11 +17,12 @@ 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.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IOContext.Context;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;

View File

@ -20,8 +20,10 @@ 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.FlushInfo;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.IOContext.Context;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
@ -90,7 +92,7 @@ final class TermVectorsTermsWriter extends TermsHashConsumer {
if (tvx == null) {
boolean success = false;
try {
IOContext context = new IOContext(Context.FLUSH);
IOContext context = new IOContext(new FlushInfo(docWriter.getNumDocsInRAM(), docWriter.bytesUsed()));
// If we hit an exception while init'ing the term
// vector output files, we must abort this segment
// because those files will be in an unknown

View File

@ -18,6 +18,7 @@ package org.apache.lucene.index;
*/
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;

View File

@ -29,7 +29,6 @@ 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;
@ -38,6 +37,7 @@ import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.index.codecs.standard.StandardPostingsReader; // javadocs
import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Bits;

View File

@ -29,6 +29,7 @@ import org.apache.lucene.index.PerDocWriteState;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.values.Writer;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.BytesRef;
public class DefaultDocValuesConsumer extends PerDocConsumer {
@ -55,7 +56,7 @@ public class DefaultDocValuesConsumer extends PerDocConsumer {
docValuesId(segmentName, codecId, field.number),
// TODO can we have a compound file per segment and codec for
// docvalues?
directory, comparator, bytesUsed);
directory, comparator, bytesUsed, IOContext.DEFAULT);
}
@SuppressWarnings("fallthrough")

View File

@ -22,7 +22,6 @@ 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;
@ -30,6 +29,7 @@ import org.apache.lucene.index.values.Floats;
import org.apache.lucene.index.values.Ints;
import org.apache.lucene.index.values.ValueType;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
/**
* Abstract base class for FieldsProducer implementations supporting
@ -57,9 +57,8 @@ public class DefaultDocValuesProducer extends PerDocValues {
* if an {@link IOException} occurs
*/
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);
FieldInfos fieldInfo, int codecId, IOContext context) throws IOException {
docValues = load(fieldInfo, si.name, si.docCount, dir, codecId, context);
}
/**
@ -73,7 +72,7 @@ public class DefaultDocValuesProducer extends PerDocValues {
// Only opens files... doesn't actually load any values
protected TreeMap<String, IndexDocValues> load(FieldInfos fieldInfos,
String segment, int docCount, Directory dir, int codecId)
String segment, int docCount, Directory dir, int codecId, IOContext context)
throws IOException {
TreeMap<String, IndexDocValues> values = new TreeMap<String, IndexDocValues>();
boolean success = false;
@ -87,7 +86,7 @@ public class DefaultDocValuesProducer extends PerDocValues {
final String id = DefaultDocValuesConsumer.docValuesId(segment,
codecId, fieldInfo.number);
values.put(field,
loadDocValues(docCount, dir, id, fieldInfo.getDocValues()));
loadDocValues(docCount, dir, id, fieldInfo.getDocValues(), context));
}
}
success = true;
@ -121,27 +120,26 @@ public class DefaultDocValuesProducer extends PerDocValues {
* if the given {@link ValueType} is not supported
*/
protected IndexDocValues loadDocValues(int docCount, Directory dir, String id,
ValueType type) throws IOException {
// nocommit this needs an IOContext too
ValueType type, IOContext context) throws IOException {
switch (type) {
case INTS:
return Ints.getValues(dir, id, false);
return Ints.getValues(dir, id, false, context);
case FLOAT_32:
return Floats.getValues(dir, id, docCount);
return Floats.getValues(dir, id, docCount, context);
case FLOAT_64:
return Floats.getValues(dir, id, docCount);
return Floats.getValues(dir, id, docCount, context);
case BYTES_FIXED_STRAIGHT:
return Bytes.getValues(dir, id, Bytes.Mode.STRAIGHT, true, docCount);
return Bytes.getValues(dir, id, Bytes.Mode.STRAIGHT, true, docCount, context);
case BYTES_FIXED_DEREF:
return Bytes.getValues(dir, id, Bytes.Mode.DEREF, true, docCount);
return Bytes.getValues(dir, id, Bytes.Mode.DEREF, true, docCount, context);
case BYTES_FIXED_SORTED:
return Bytes.getValues(dir, id, Bytes.Mode.SORTED, true, docCount);
return Bytes.getValues(dir, id, Bytes.Mode.SORTED, true, docCount, context);
case BYTES_VAR_STRAIGHT:
return Bytes.getValues(dir, id, Bytes.Mode.STRAIGHT, false, docCount);
return Bytes.getValues(dir, id, Bytes.Mode.STRAIGHT, false, docCount, context);
case BYTES_VAR_DEREF:
return Bytes.getValues(dir, id, Bytes.Mode.DEREF, false, docCount);
return Bytes.getValues(dir, id, Bytes.Mode.DEREF, false, docCount, context);
case BYTES_VAR_SORTED:
return Bytes.getValues(dir, id, Bytes.Mode.SORTED, false, docCount);
return Bytes.getValues(dir, id, Bytes.Mode.SORTED, false, docCount, context);
default:
throw new IllegalStateException("unrecognized index values mode " + type);
}

View File

@ -22,15 +22,14 @@ 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.IOContext;
import org.apache.lucene.store.IndexInput;
/**

View File

@ -19,12 +19,11 @@ 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.IOContext;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.IOUtils;

View File

@ -18,11 +18,11 @@ package org.apache.lucene.index.codecs;
*/
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IOContext.Context;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.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;

View File

@ -19,9 +19,9 @@ 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.IOContext;
/**
* Specifies an API for classes that can read {@link SegmentInfos} information.

View File

@ -19,9 +19,9 @@ 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.IOContext;
import org.apache.lucene.store.IndexOutput;
/**

View File

@ -27,12 +27,12 @@ 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;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IOContext.Context;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CodecUtil;
import org.apache.lucene.util.fst.Builder;

View File

@ -48,6 +48,7 @@ import org.apache.lucene.index.codecs.TermStats;
import org.apache.lucene.index.codecs.TermsConsumer;
import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.RAMOutputStream;
@ -697,9 +698,8 @@ 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, state.context);
final IndexInput in = state.dir.openInput(fileName, IOContext.READONCE);
final SortedMap<String,TermsReader> fields = new TreeMap<String,TermsReader>();
@ -776,6 +776,6 @@ public class MemoryCodec extends Codec {
@Override
public PerDocValues docsProducer(SegmentReadState state) throws IOException {
return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId);
return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId, IOContext.READONCE);
}
}

View File

@ -31,7 +31,6 @@ 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;
@ -39,6 +38,7 @@ import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.index.codecs.FieldsProducer;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;

View File

@ -21,10 +21,10 @@ 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;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.CloseableThreadLocal;
import org.apache.lucene.util.DoubleBarrelLRUCache;

View File

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

View File

@ -17,9 +17,8 @@ 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;
import org.apache.lucene.store.IOContext;
import java.io.IOException;
@ -30,5 +29,5 @@ public abstract class IntStreamFactory {
}
// public abstract IntIndexInput openInput(Directory dir, String fileName, IOContext context) throws IOException;
public abstract IntIndexOutput createOutput(Directory dir, String fileName) throws IOException;
public abstract IntIndexOutput createOutput(Directory dir, String fileName, IOContext context) throws IOException;
}

View File

@ -23,7 +23,6 @@ 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;
@ -31,6 +30,7 @@ import org.apache.lucene.index.codecs.BlockTermState;
import org.apache.lucene.index.codecs.PostingsReaderBase;
import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Bits;

View File

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

View File

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

View File

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

View File

@ -23,7 +23,6 @@ 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;
@ -31,6 +30,7 @@ import org.apache.lucene.index.codecs.BlockTermState;
import org.apache.lucene.index.codecs.PostingsReaderBase;
import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Bits;

View File

@ -23,12 +23,12 @@ 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;
import org.apache.lucene.index.values.IndexDocValues.SourceEnum;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.AttributeSource;
@ -107,9 +107,8 @@ public final class Bytes {
* if the files for the writer can not be created.
*/
public static Writer getWriter(Directory dir, String id, Mode mode,
Comparator<BytesRef> comp, boolean fixedSize, AtomicLong bytesUsed)
Comparator<BytesRef> comp, boolean fixedSize, AtomicLong bytesUsed, IOContext context)
throws IOException {
//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?
@ -119,19 +118,19 @@ public final class Bytes {
if (fixedSize) {
if (mode == Mode.STRAIGHT) {
return new FixedStraightBytesImpl.Writer(dir, id);
return new FixedStraightBytesImpl.Writer(dir, id, context);
} else if (mode == Mode.DEREF) {
return new FixedDerefBytesImpl.Writer(dir, id, bytesUsed);
return new FixedDerefBytesImpl.Writer(dir, id, bytesUsed, context);
} else if (mode == Mode.SORTED) {
return new FixedSortedBytesImpl.Writer(dir, id, comp, bytesUsed);
return new FixedSortedBytesImpl.Writer(dir, id, comp, bytesUsed, context);
}
} else {
if (mode == Mode.STRAIGHT) {
return new VarStraightBytesImpl.Writer(dir, id, bytesUsed);
return new VarStraightBytesImpl.Writer(dir, id, bytesUsed, context);
} else if (mode == Mode.DEREF) {
return new VarDerefBytesImpl.Writer(dir, id, bytesUsed);
return new VarDerefBytesImpl.Writer(dir, id, bytesUsed, context);
} else if (mode == Mode.SORTED) {
return new VarSortedBytesImpl.Writer(dir, id, comp, bytesUsed);
return new VarSortedBytesImpl.Writer(dir, id, comp, bytesUsed, context);
}
}
@ -160,25 +159,24 @@ public final class Bytes {
* if an {@link IOException} occurs
*/
public static IndexDocValues getValues(Directory dir, String id, Mode mode,
boolean fixedSize, int maxDoc) throws IOException {
//nocommit this and all the readers below need an IOContext too
boolean fixedSize, int maxDoc, IOContext context) throws IOException {
// TODO -- I can peek @ header to determing fixed/mode?
if (fixedSize) {
if (mode == Mode.STRAIGHT) {
return new FixedStraightBytesImpl.Reader(dir, id, maxDoc);
return new FixedStraightBytesImpl.Reader(dir, id, maxDoc, context);
} else if (mode == Mode.DEREF) {
return new FixedDerefBytesImpl.Reader(dir, id, maxDoc);
return new FixedDerefBytesImpl.Reader(dir, id, maxDoc, context);
} else if (mode == Mode.SORTED) {
return new FixedSortedBytesImpl.Reader(dir, id, maxDoc);
return new FixedSortedBytesImpl.Reader(dir, id, maxDoc, context);
}
} else {
if (mode == Mode.STRAIGHT) {
return new VarStraightBytesImpl.Reader(dir, id, maxDoc);
return new VarStraightBytesImpl.Reader(dir, id, maxDoc, context);
} else if (mode == Mode.DEREF) {
return new VarDerefBytesImpl.Reader(dir, id, maxDoc);
return new VarDerefBytesImpl.Reader(dir, id, maxDoc, context);
} else if (mode == Mode.SORTED) {
return new VarSortedBytesImpl.Reader(dir, id, maxDoc);
return new VarSortedBytesImpl.Reader(dir, id, maxDoc, context);
}
}

View File

@ -20,11 +20,11 @@ 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;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.AttributeSource;
@ -55,17 +55,16 @@ class FixedDerefBytesImpl {
private final BytesRefHash hash = new BytesRefHash(pool,
BytesRefHash.DEFAULT_CAPACITY, new TrackingDirectBytesStartArray(
BytesRefHash.DEFAULT_CAPACITY, bytesUsed));
public Writer(Directory dir, String id, AtomicLong bytesUsed)
public Writer(Directory dir, String id, AtomicLong bytesUsed, IOContext context)
throws IOException {
//nocommit this needs an IOContext too
this(dir, id, new DirectTrackingAllocator(ByteBlockPool.BYTE_BLOCK_SIZE, bytesUsed),
bytesUsed);
bytesUsed, context);
}
public Writer(Directory dir, String id, Allocator allocator,
AtomicLong bytesUsed) throws IOException {
AtomicLong bytesUsed, IOContext context) throws IOException {
super(dir, id, CODEC_NAME, VERSION_CURRENT, true,
new ByteBlockPool(allocator), bytesUsed, IOContext.DEFAULT);
new ByteBlockPool(allocator), bytesUsed, context);
docToID = new int[1];
bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT); // TODO BytesRefHash
// uses bytes too!
@ -135,8 +134,8 @@ class FixedDerefBytesImpl {
public static class Reader extends BytesReaderBase {
private final int size;
Reader(Directory dir, String id, int maxDoc) throws IOException {
super(dir, id, CODEC_NAME, VERSION_START, true, IOContext.DEFAULT);
Reader(Directory dir, String id, int maxDoc, IOContext context) throws IOException {
super(dir, id, CODEC_NAME, VERSION_START, true, context);
size = datIn.readInt();
}

View File

@ -21,12 +21,12 @@ 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;
import org.apache.lucene.index.values.FixedDerefBytesImpl.Reader.DerefBytesEnum;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.AttributeSource;
@ -63,16 +63,15 @@ class FixedSortedBytesImpl {
BytesRefHash.DEFAULT_CAPACITY, bytesUsed));
public Writer(Directory dir, String id, Comparator<BytesRef> comp,
AtomicLong bytesUsed) throws IOException {
//nocommit this needs an IOContext too
AtomicLong bytesUsed, IOContext context) throws IOException {
this(dir, id, comp, new DirectTrackingAllocator(ByteBlockPool.BYTE_BLOCK_SIZE, bytesUsed),
bytesUsed);
bytesUsed, context);
}
public Writer(Directory dir, String id, Comparator<BytesRef> comp,
Allocator allocator, AtomicLong bytesUsed) throws IOException {
Allocator allocator, AtomicLong bytesUsed, IOContext context) throws IOException {
super(dir, id, CODEC_NAME, VERSION_CURRENT, true,
new ByteBlockPool(allocator), bytesUsed, IOContext.DEFAULT);
new ByteBlockPool(allocator), bytesUsed, context);
docToEntry = new int[1];
// docToEntry[0] = -1;
bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT);
@ -162,8 +161,8 @@ class FixedSortedBytesImpl {
public static class Reader extends BytesReaderBase {
private final int size;
public Reader(Directory dir, String id, int maxDoc) throws IOException {
super(dir, id, CODEC_NAME, VERSION_START, true, IOContext.DEFAULT);
public Reader(Directory dir, String id, int maxDoc, IOContext context) throws IOException {
super(dir, id, CODEC_NAME, VERSION_START, true, context);
size = datIn.readInt();
}

View File

@ -19,11 +19,11 @@ 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;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.AttributeSource;
import org.apache.lucene.util.BytesRef;
@ -47,9 +47,8 @@ class FixedStraightBytesImpl {
private int lastDocID = -1;
private byte[] oneRecord;
public Writer(Directory dir, String id) throws IOException {
//nocommit this needs an IOContext too
super(dir, id, CODEC_NAME, VERSION_CURRENT, false, null, null, IOContext.DEFAULT);
public Writer(Directory dir, String id, IOContext context) throws IOException {
super(dir, id, CODEC_NAME, VERSION_CURRENT, false, null, null, context);
}
@ -127,8 +126,8 @@ class FixedStraightBytesImpl {
private final int size;
private final int maxDoc;
Reader(Directory dir, String id, int maxDoc) throws IOException {
super(dir, id, CODEC_NAME, VERSION_START, false, IOContext.DEFAULT);
Reader(Directory dir, String id, int maxDoc, IOContext context) throws IOException {
super(dir, id, CODEC_NAME, VERSION_START, false, context);
size = datIn.readInt();
this.maxDoc = maxDoc;
}

View File

@ -20,11 +20,10 @@ 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;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.AttributeSource;
@ -53,22 +52,21 @@ public class Floats {
public static Writer getWriter(Directory dir, String id, int precisionBytes,
AtomicLong bytesUsed) throws IOException {
AtomicLong bytesUsed, IOContext context) throws IOException {
if (precisionBytes != 4 && precisionBytes != 8) {
throw new IllegalArgumentException("precisionBytes must be 4 or 8; got "
+ precisionBytes);
}
if (precisionBytes == 4) {
return new Float4Writer(dir, id, bytesUsed);
return new Float4Writer(dir, id, bytesUsed, context);
} else {
return new Float8Writer(dir, id, bytesUsed);
return new Float8Writer(dir, id, bytesUsed, context);
}
}
public static IndexDocValues getValues(Directory dir, String id, int maxDoc)
public static IndexDocValues getValues(Directory dir, String id, int maxDoc, IOContext context)
throws IOException {
//nocommit this needs an IOContext too
return new FloatsReader(dir, id, maxDoc, IOContext.READ);
return new FloatsReader(dir, id, maxDoc, context);
}
abstract static class FloatsWriter extends Writer {
@ -147,9 +145,9 @@ public class Floats {
// Writes 4 bytes (float) per value
static class Float4Writer extends FloatsWriter {
protected Float4Writer(Directory dir, String id, AtomicLong bytesUsed)
protected Float4Writer(Directory dir, String id, AtomicLong bytesUsed, IOContext context)
throws IOException {
super(dir, id, 4, bytesUsed, new IOContext(Context.FLUSH));
super(dir, id, 4, bytesUsed, context);
}
@Override
@ -190,9 +188,9 @@ public class Floats {
// Writes 8 bytes (double) per value
static class Float8Writer extends FloatsWriter {
protected Float8Writer(Directory dir, String id, AtomicLong bytesUsed)
protected Float8Writer(Directory dir, String id, AtomicLong bytesUsed, IOContext context)
throws IOException {
super(dir, id, 8, bytesUsed, new IOContext(Context.FLUSH));
super(dir, id, 8, bytesUsed, context);
}
@Override

View File

@ -20,10 +20,10 @@ 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;
import org.apache.lucene.store.IOContext;
/**
* @lucene.experimental
@ -35,14 +35,13 @@ public class Ints {
}
public static Writer getWriter(Directory dir, String id,
boolean useFixedArray, AtomicLong bytesUsed) throws IOException {
boolean useFixedArray, AtomicLong bytesUsed, IOContext context) throws IOException {
// TODO - implement fixed?!
return new IntsWriter(dir, id, bytesUsed, IOContext.DEFAULT);
return new IntsWriter(dir, id, bytesUsed, context);
}
public static IndexDocValues getValues(Directory dir, String id,
boolean useFixedArray) throws IOException {
//nocommit this needs an IOContext too
return new IntsReader(dir, id, IOContext.DEFAULT);
boolean useFixedArray, IOContext context) throws IOException {
return new IntsReader(dir, id, context);
}
}

View File

@ -20,9 +20,9 @@ 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.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.ArrayUtil;

View File

@ -20,13 +20,13 @@ 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;
import org.apache.lucene.index.values.FixedDerefBytesImpl.Reader.DerefBytesEnum;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.AttributeSource;
@ -116,17 +116,16 @@ class VarDerefBytesImpl {
bytesUsed);
private final BytesRefHash hash = new BytesRefHash(pool, 16, array);
public Writer(Directory dir, String id, AtomicLong bytesUsed)
public Writer(Directory dir, String id, AtomicLong bytesUsed, IOContext context)
throws IOException {
//nocommit this needs an IOContext too
this(dir, id, new DirectTrackingAllocator(ByteBlockPool.BYTE_BLOCK_SIZE, bytesUsed),
bytesUsed);
bytesUsed, context);
}
public Writer(Directory dir, String id, Allocator allocator,
AtomicLong bytesUsed) throws IOException {
AtomicLong bytesUsed, IOContext context) throws IOException {
super(dir, id, CODEC_NAME, VERSION_CURRENT, true,
new ByteBlockPool(allocator), bytesUsed, IOContext.DEFAULT);
new ByteBlockPool(allocator), bytesUsed, context);
docToAddress = new int[1];
bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT);
}
@ -203,8 +202,8 @@ class VarDerefBytesImpl {
public static class Reader extends BytesReaderBase {
Reader(Directory dir, String id, int maxDoc) throws IOException {
super(dir, id, CODEC_NAME, VERSION_START, true, IOContext.DEFAULT);
Reader(Directory dir, String id, int maxDoc, IOContext context) throws IOException {
super(dir, id, CODEC_NAME, VERSION_START, true, context);
}
@Override

View File

@ -22,11 +22,11 @@ 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;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.AttributeSource;
@ -62,16 +62,15 @@ class VarSortedBytesImpl {
BytesRefHash.DEFAULT_CAPACITY, bytesUsed));
public Writer(Directory dir, String id, Comparator<BytesRef> comp,
AtomicLong bytesUsed) throws IOException {
//nocommit this needs an IOContext too
AtomicLong bytesUsed, IOContext context) throws IOException {
this(dir, id, comp, new DirectTrackingAllocator(ByteBlockPool.BYTE_BLOCK_SIZE, bytesUsed),
bytesUsed);
bytesUsed, context);
}
public Writer(Directory dir, String id, Comparator<BytesRef> comp,
Allocator allocator, AtomicLong bytesUsed) throws IOException {
Allocator allocator, AtomicLong bytesUsed, IOContext context) throws IOException {
super(dir, id, CODEC_NAME, VERSION_CURRENT, true,
new ByteBlockPool(allocator), bytesUsed, IOContext.DEFAULT);
new ByteBlockPool(allocator), bytesUsed, context);
this.comp = comp;
docToEntry = new int[1];
docToEntry[0] = -1;
@ -158,8 +157,8 @@ 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, IOContext.DEFAULT);
Reader(Directory dir, String id, int maxDoc, IOContext context) throws IOException {
super(dir, id, CODEC_NAME, VERSION_START, true, context);
}
@Override

View File

@ -20,11 +20,11 @@ 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;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.AttributeSource;
@ -50,16 +50,15 @@ class VarStraightBytesImpl {
private int lastDocID = -1;
private long[] docToAddress;
public Writer(Directory dir, String id, AtomicLong bytesUsed)
public Writer(Directory dir, String id, AtomicLong bytesUsed, IOContext context)
throws IOException {
//nocommit this needs an IOContext too
super(dir, id, CODEC_NAME, VERSION_CURRENT, true, null, bytesUsed, IOContext.DEFAULT);
super(dir, id, CODEC_NAME, VERSION_CURRENT, true, null, bytesUsed, context);
docToAddress = new long[1];
bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT);
}
public Writer(Directory dir, String id) throws IOException {
this(dir, id, new AtomicLong());
public Writer(Directory dir, String id, IOContext context) throws IOException {
this(dir, id, new AtomicLong(), context);
}
// Fills up to but not including this docID
@ -123,8 +122,8 @@ class VarStraightBytesImpl {
public static class Reader extends BytesReaderBase {
private final int maxDoc;
Reader(Directory dir, String id, int maxDoc) throws IOException {
super(dir, id, CODEC_NAME, VERSION_START, true, IOContext.DEFAULT);
Reader(Directory dir, String id, int maxDoc, IOContext context) throws IOException {
super(dir, id, CODEC_NAME, VERSION_START, true, context);
this.maxDoc = maxDoc;
}

View File

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

View File

@ -22,7 +22,6 @@ 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

View File

@ -29,7 +29,6 @@ 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;

View File

@ -25,7 +25,6 @@ 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

View File

@ -0,0 +1,45 @@
package org.apache.lucene.store;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
/**
* <p>A FlushInfo provides information required for a FLUSH context and other optimization operations.
* It is used as part of an {@link IOContext} in case of FLUSH context.</p>
*/
public class FlushInfo {
public final int numDocs;
public final long estimatedSegmentSize;
/**
* <p>Creates a new {@link FlushInfo} instance from
* the values required for a FLUSH {@link IOContext} context.
*
* These values are only estimates and are not the actual values.
*
*/
public FlushInfo(int numDocs, long estimatedSegmentSize) {
this.numDocs = numDocs;
this.estimatedSegmentSize = estimatedSegmentSize;
}
}

View File

@ -1,4 +1,4 @@
package org.apache.lucene.index;
package org.apache.lucene.store;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -37,6 +37,8 @@ public class IOContext {
public final MergeInfo mergeInfo;
public final FlushInfo flushInfo;
public final boolean readOnce;
public static final IOContext DEFAULT = new IOContext(Context.DEFAULT);
@ -49,14 +51,23 @@ public class IOContext {
this(false);
}
public IOContext(Context context) {
public IOContext (FlushInfo flushInfo) {
assert flushInfo != null;
this.context = Context.FLUSH;
this.mergeInfo = null;
this.readOnce = false;
this.flushInfo = flushInfo;
}
public IOContext (Context context) {
this(context, null);
}
private IOContext(boolean readOnce) {
private IOContext (boolean readOnce) {
this.context = Context.READ;
this.mergeInfo = null;
this.readOnce = readOnce;
this.flushInfo = null;
}
public IOContext (MergeInfo mergeInfo) {
@ -64,10 +75,11 @@ public class IOContext {
}
private IOContext (Context context, MergeInfo mergeInfo ) {
assert context != Context.MERGE || mergeInfo != null;
assert context != Context.MERGE || context != Context.FLUSH || mergeInfo != null;
this.context = context;
this.readOnce = false;
this.mergeInfo = mergeInfo;
this.flushInfo = null;
}
}

View File

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

View File

@ -1,4 +1,4 @@
package org.apache.lucene.index;
package org.apache.lucene.store;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
@ -15,7 +15,12 @@ package org.apache.lucene.index;
* See the License for the specific language governing permissions and
* limitations under the License.
*/
//nocommit javadoc
/**
* <p>A MergeInfo provides information required for a MERGE context and other optimization operations.
* It is used as part of an {@link IOContext} in case of MERGE context.</p>
*/
public class MergeInfo {
public int totalDocCount;
@ -25,6 +30,15 @@ public class MergeInfo {
boolean isExternal; // used by IndexWriter
boolean optimize; // used by IndexWriter
/**
* <p>Creates a new {@link MergeInfo} instance from
* the values required for a MERGE {@link IOContext} context.
*
* These values are only estimates and are not the actual values.
*
*/
public MergeInfo(int totalDocCount, long estimatedMergeBytes, boolean isExternal, boolean optimize) {
this.totalDocCount = totalDocCount;

View File

@ -23,7 +23,6 @@ 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

View File

@ -27,7 +27,6 @@ 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

View File

@ -21,7 +21,6 @@ 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

View File

@ -19,12 +19,12 @@ 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.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.IOContext.Context;
/** Optimized implementation of a vector of bits. This is more-or-less like
java.util.BitSet, but also includes the following:

View File

@ -20,7 +20,6 @@ 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;
@ -103,8 +102,8 @@ public class MockFixedIntBlockCodec extends Codec {
}
@Override
public IntIndexOutput createOutput(Directory dir, String fileName) throws IOException {
IndexOutput out = dir.createOutput(fileName, IOContext.DEFAULT);
public IntIndexOutput createOutput(Directory dir, String fileName, IOContext context) throws IOException {
IndexOutput out = dir.createOutput(fileName, context);
boolean success = false;
try {
FixedIntBlockIndexOutput ret = new FixedIntBlockIndexOutput(out, blockSize) {
@ -226,6 +225,6 @@ public class MockFixedIntBlockCodec extends Codec {
@Override
public PerDocValues docsProducer(SegmentReadState state) throws IOException {
return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId);
return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId, state.context);
}
}

View File

@ -20,7 +20,6 @@ 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;
@ -49,6 +48,7 @@ import org.apache.lucene.index.codecs.TermsIndexReaderBase;
import org.apache.lucene.index.codecs.TermsIndexWriterBase;
import org.apache.lucene.index.codecs.standard.StandardCodec;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
@ -107,8 +107,8 @@ public class MockVariableIntBlockCodec extends Codec {
}
@Override
public IntIndexOutput createOutput(Directory dir, String fileName) throws IOException {
final IndexOutput out = dir.createOutput(fileName, IOContext.DEFAULT);
public IntIndexOutput createOutput(Directory dir, String fileName, IOContext context) throws IOException {
final IndexOutput out = dir.createOutput(fileName, context);
boolean success = false;
try {
out.writeInt(baseBlockSize);
@ -248,6 +248,6 @@ public class MockVariableIntBlockCodec extends Codec {
@Override
public PerDocValues docsProducer(SegmentReadState state) throws IOException {
return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId);
return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId, state.context);
}
}

View File

@ -25,7 +25,6 @@ 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;
@ -62,6 +61,7 @@ import org.apache.lucene.index.codecs.sep.SepPostingsWriterImpl;
import org.apache.lucene.index.codecs.standard.StandardPostingsReader;
import org.apache.lucene.index.codecs.standard.StandardPostingsWriter;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
@ -115,12 +115,12 @@ public class MockRandomCodec extends Codec {
}
@Override
public IntIndexOutput createOutput(Directory dir, String fileName) throws IOException {
public IntIndexOutput createOutput(Directory dir, String fileName, IOContext context) throws IOException {
final IntStreamFactory f = delegates.get((Math.abs(salt ^ getExtension(fileName).hashCode())) % delegates.size());
if (LuceneTestCase.VERBOSE) {
System.out.println("MockRandomCodec: write using int factory " + f + " to fileName=" + fileName);
}
return f.createOutput(dir, fileName);
return f.createOutput(dir, fileName, context);
}
}
@ -386,6 +386,6 @@ public class MockRandomCodec extends Codec {
@Override
public PerDocValues docsProducer(SegmentReadState state) throws IOException {
return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId);
return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId, state.context);
}
}

View File

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

View File

@ -18,7 +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.store.IOContext;
import org.apache.lucene.index.codecs.sep.IntStreamFactory;
import org.apache.lucene.index.codecs.sep.IntIndexInput;
import org.apache.lucene.index.codecs.sep.IntIndexOutput;
@ -32,7 +32,7 @@ public class MockSingleIntFactory extends IntStreamFactory {
return new MockSingleIntIndexInput(dir, fileName, context);
}
@Override
public IntIndexOutput createOutput(Directory dir, String fileName) throws IOException {
return new MockSingleIntIndexOutput(dir, fileName);
public IntIndexOutput createOutput(Directory dir, String fileName, IOContext context) throws IOException {
return new MockSingleIntIndexOutput(dir, fileName, context);
}
}

View File

@ -19,10 +19,10 @@ 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;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.CodecUtil;

View File

@ -17,11 +17,11 @@ package org.apache.lucene.index.codecs.mocksep;
* limitations under the License.
*/
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.CodecUtil;
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,9 +36,8 @@ public class MockSingleIntIndexOutput extends IntIndexOutput {
final static int VERSION_START = 0;
final static int VERSION_CURRENT = VERSION_START;
public MockSingleIntIndexOutput(Directory dir, String fileName) throws IOException {
//nocommit pass IOContext in via ctor!
out = dir.createOutput(fileName, IOContext.DEFAULT);
public MockSingleIntIndexOutput(Directory dir, String fileName, IOContext context) throws IOException {
out = dir.createOutput(fileName, context);
boolean success = false;
try {
CodecUtil.writeHeader(out, CODEC, VERSION_CURRENT);

View File

@ -22,11 +22,11 @@ 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;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CharsRef;

View File

@ -31,7 +31,6 @@ 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;
@ -198,7 +197,7 @@ public class MockDirectoryWrapper extends Directory {
byte[] zeroes = new byte[256];
long upto = 0;
//nocommit - randomize the IOContext here?
IndexOutput out = delegate.createOutput(name, IOContext.DEFAULT);
IndexOutput out = delegate.createOutput(name, LuceneTestCase.newIOContext(randomState));
while(upto < length) {
final int limit = (int) Math.min(length-upto, zeroes.length);
out.writeBytes(zeroes, 0, limit);
@ -208,7 +207,7 @@ public class MockDirectoryWrapper extends Directory {
} else if (count % 3 == 2) {
// Truncate the file:
//nocommit - randomize the IOContext here?
IndexOutput out = delegate.createOutput(name, IOContext.DEFAULT);
IndexOutput out = delegate.createOutput(name, LuceneTestCase.newIOContext(randomState));
out.setLength(fileLength(name)/2);
out.close();
}
@ -376,7 +375,7 @@ public class MockDirectoryWrapper extends Directory {
//System.out.println(Thread.currentThread().getName() + ": MDW: create " + name);
// nocommit - randomize the IOContext here?
IndexOutput io = new MockIndexOutputWrapper(this, delegate.createOutput(name, context), name);
IndexOutput io = new MockIndexOutputWrapper(this, delegate.createOutput(name, LuceneTestCase.newIOContext(randomState)), name);
addFileHandle(io, name, false);
openFilesForWrite.add(name);
@ -417,7 +416,7 @@ public class MockDirectoryWrapper extends Directory {
}
// nocommit - randomize IOContext here?
IndexInput ii = new MockIndexInputWrapper(this, name, delegate.openInput(name, context));
IndexInput ii = new MockIndexInputWrapper(this, name, delegate.openInput(name, LuceneTestCase.newIOContext(randomState)));
addFileHandle(ii, name, true);
return ii;
}

View File

@ -60,7 +60,10 @@ import org.apache.lucene.search.AssertingIndexSearcher;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FSDirectory;
import org.apache.lucene.store.FlushInfo;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.LockFactory;
import org.apache.lucene.store.MergeInfo;
import org.apache.lucene.store.MockDirectoryWrapper;
import org.apache.lucene.store.MockDirectoryWrapper.Throttling;
import org.apache.lucene.util.FieldCacheSanityChecker.Insanity;
@ -1312,6 +1315,38 @@ public abstract class LuceneTestCase extends Assert {
return sb.toString();
}
public static IOContext newIOContext(Random random) {
int randomNumDocs=4192, size=2048;
int type = random.nextInt(5);
IOContext context = IOContext.DEFAULT;
switch(type) {
case 0:
context = IOContext.DEFAULT;
break;
case 1:
context = IOContext.READ;
break;
case 2:
context = IOContext.READONCE;
break;
case 3:
randomNumDocs = random.nextInt(4192);
size = random.nextInt(2048);
context = new IOContext(new MergeInfo(randomNumDocs, size, true, false));
break;
case 4:
randomNumDocs = random.nextInt(4192);
size = random.nextInt(2048);
context = new IOContext(new FlushInfo(randomNumDocs, size));
break;
}
return context;
}
// recorded seed: for beforeClass
private static long staticSeed;
// seed for individual test methods, changed in @before

View File

@ -38,6 +38,7 @@ import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.PhraseQuery;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.MockDirectoryWrapper;
import org.apache.lucene.store.RAMDirectory;
import org.apache.lucene.util.LuceneTestCase;
@ -395,7 +396,7 @@ public class TestAddIndexes extends LuceneTestCase {
setMergePolicy(newLogMergePolicy(4))
);
writer.addIndexes(aux, new MockDirectoryWrapper(random, new RAMDirectory(aux, IOContext.DEFAULT)));
writer.addIndexes(aux, new MockDirectoryWrapper(random, new RAMDirectory(aux, newIOContext(random))));
assertEquals(1060, writer.maxDoc());
assertEquals(1000, writer.getDocCount(0));
writer.close();
@ -430,7 +431,7 @@ public class TestAddIndexes extends LuceneTestCase {
setMergePolicy(newLogMergePolicy(4))
);
writer.addIndexes(aux, new MockDirectoryWrapper(random, new RAMDirectory(aux, IOContext.DEFAULT)));
writer.addIndexes(aux, new MockDirectoryWrapper(random, new RAMDirectory(aux, newIOContext(random))));
assertEquals(1020, writer.maxDoc());
assertEquals(1000, writer.getDocCount(0));
writer.close();
@ -665,7 +666,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, IOContext.DEFAULT));
dirs[k] = new MockDirectoryWrapper(random, new RAMDirectory(dir, newIOContext(random)));
int j=0;

View File

@ -31,7 +31,6 @@ 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;
@ -43,7 +42,9 @@ import org.apache.lucene.search.Similarity;
import org.apache.lucene.search.SimilarityProvider;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.RAMDirectory;
import org.apache.lucene.store.IOContext.Context;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
@ -537,7 +538,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", IOContext.READ);
CompoundFileReader cfsReader = new CompoundFileReader(dir, "_0.cfs", newIOContext(random));
FieldInfos fieldInfos = new FieldInfos(cfsReader, "_0.fnm");
int contentFieldIndex = -1;
for (FieldInfo fi : fieldInfos) {

View File

@ -25,7 +25,6 @@ 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;
@ -40,6 +39,8 @@ import org.apache.lucene.search.PhraseQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IOContext.Context;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.Version;
@ -243,7 +244,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, IOContext.READ, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR));
final FieldsProducer reader = si.getSegmentCodecs().codec().fieldsProducer(new SegmentReadState(dir, si, fieldInfos, newIOContext(random), IndexReader.DEFAULT_TERMS_INDEX_DIVISOR));
final FieldsEnum fieldsEnum = reader.iterator();
assertNotNull(fieldsEnum.next());
@ -298,7 +299,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, IOContext.READ, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR));
final FieldsProducer terms = si.getSegmentCodecs().codec().fieldsProducer(new SegmentReadState(dir, si, fieldInfos, newIOContext(random), IndexReader.DEFAULT_TERMS_INDEX_DIVISOR));
final Verify[] threads = new Verify[NUM_TEST_THREADS-1];
for(int i=0;i<NUM_TEST_THREADS-1;i++) {
@ -592,7 +593,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, new IOContext(Context.FLUSH));
final SegmentWriteState state = new SegmentWriteState(null, dir, SEGMENT, fieldInfos, 10000, termIndexInterval, codecInfo, null, newIOContext(random));
final FieldsConsumer consumer = state.segmentCodecs.codec().fieldsConsumer(state);
Arrays.sort(fields);

View File

@ -24,12 +24,13 @@ 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.IOContext;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.SimpleFSDirectory;
import org.apache.lucene.store._TestHelper;
import org.apache.lucene.store.IOContext.Context;
import org.apache.lucene.util._TestUtil;
@ -74,7 +75,7 @@ public class TestCompoundFile extends LuceneTestCase
private void createRandomFile(Directory dir, String name, int size)
throws IOException
{
IndexOutput os = dir.createOutput(name, new IOContext(Context.FLUSH));
IndexOutput os = dir.createOutput(name, newIOContext(random));
for (int i=0; i<size; i++) {
byte b = (byte) (Math.random() * 256);
os.writeByte(b);
@ -92,7 +93,7 @@ public class TestCompoundFile extends LuceneTestCase
int size)
throws IOException
{
IndexOutput os = dir.createOutput(name, new IOContext(Context.FLUSH));
IndexOutput os = dir.createOutput(name, newIOContext(random));
for (int i=0; i < size; i++) {
os.writeByte(start);
start ++;
@ -202,13 +203,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", IOContext.DEFAULT);
CompoundFileWriter csw = new CompoundFileWriter(dir, name + ".cfs", newIOContext(random));
csw.addFile(name);
csw.close();
CompoundFileReader csr = new CompoundFileReader(dir, name + ".cfs", IOContext.DEFAULT);
IndexInput expected = dir.openInput(name, IOContext.DEFAULT);
IndexInput actual = csr.openInput(name, IOContext.DEFAULT);
CompoundFileReader csr = new CompoundFileReader(dir, name + ".cfs", newIOContext(random));
IndexInput expected = dir.openInput(name, newIOContext(random));
IndexInput actual = csr.openInput(name, newIOContext(random));
assertSameStreams(name, expected, actual);
assertSameSeekBehavior(name, expected, actual);
expected.close();
@ -225,21 +226,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", IOContext.DEFAULT);
CompoundFileWriter csw = new CompoundFileWriter(dir, "d.csf", newIOContext(random));
csw.addFile("d1");
csw.addFile("d2");
csw.close();
CompoundFileReader csr = new CompoundFileReader(dir, "d.csf", IOContext.DEFAULT);
IndexInput expected = dir.openInput("d1", IOContext.DEFAULT);
IndexInput actual = csr.openInput("d1", IOContext.DEFAULT);
CompoundFileReader csr = new CompoundFileReader(dir, "d.csf", newIOContext(random));
IndexInput expected = dir.openInput("d1", newIOContext(random));
IndexInput actual = csr.openInput("d1", newIOContext(random));
assertSameStreams("d1", expected, actual);
assertSameSeekBehavior("d1", expected, actual);
expected.close();
actual.close();
expected = dir.openInput("d2", IOContext.DEFAULT);
actual = csr.openInput("d2", IOContext.DEFAULT);
expected = dir.openInput("d2", newIOContext(random));
actual = csr.openInput("d2", newIOContext(random));
assertSameStreams("d2", expected, actual);
assertSameSeekBehavior("d2", expected, actual);
expected.close();
@ -275,7 +276,7 @@ public class TestCompoundFile extends LuceneTestCase
createRandomFile(dir, segment + ".notIn2", 51);
// Now test
CompoundFileWriter csw = new CompoundFileWriter(dir, "test.cfs", IOContext.DEFAULT);
CompoundFileWriter csw = new CompoundFileWriter(dir, "test.cfs", newIOContext(random));
final String data[] = new String[] {
".zero", ".one", ".ten", ".hundred", ".big1", ".big2", ".big3",
".big4", ".big5", ".big6", ".big7"
@ -285,10 +286,10 @@ public class TestCompoundFile extends LuceneTestCase
}
csw.close();
CompoundFileReader csr = new CompoundFileReader(dir, "test.cfs", IOContext.DEFAULT);
CompoundFileReader csr = new CompoundFileReader(dir, "test.cfs", newIOContext(random));
for (int i=0; i<data.length; i++) {
IndexInput check = dir.openInput(segment + data[i], IOContext.DEFAULT);
IndexInput test = csr.openInput(segment + data[i], IOContext.DEFAULT);
IndexInput check = dir.openInput(segment + data[i], newIOContext(random));
IndexInput test = csr.openInput(segment + data[i], newIOContext(random));
assertSameStreams(data[i], check, test);
assertSameSeekBehavior(data[i], check, test);
test.close();
@ -304,7 +305,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", IOContext.DEFAULT);
CompoundFileWriter cw = new CompoundFileWriter(dir, "f.comp", newIOContext(random));
for (int i=0; i<20; i++) {
createSequenceFile(dir, "f" + i, (byte) 0, 2000);
cw.addFile("f" + i);
@ -321,13 +322,13 @@ public class TestCompoundFile extends LuceneTestCase
throws IOException
{
// Setup the test file - we need more than 1024 bytes
IndexOutput os = fsdir.createOutput(file, IOContext.DEFAULT);
IndexOutput os = fsdir.createOutput(file, newIOContext(random));
for(int i=0; i<2000; i++) {
os.writeByte((byte) i);
}
os.close();
IndexInput in = fsdir.openInput(file, IOContext.DEFAULT);
IndexInput in = fsdir.openInput(file, newIOContext(random));
// This read primes the buffer in IndexInput
in.readByte();
@ -371,16 +372,16 @@ public class TestCompoundFile extends LuceneTestCase
public void testClonedStreamsClosing() throws IOException {
setUp_2();
CompoundFileReader cr = new CompoundFileReader(dir, "f.comp", IOContext.DEFAULT);
CompoundFileReader cr = new CompoundFileReader(dir, "f.comp", newIOContext(random));
// basic clone
IndexInput expected = dir.openInput("f11", IOContext.DEFAULT);
IndexInput expected = dir.openInput("f11", newIOContext(random));
// this test only works for FSIndexInput
assertTrue(_TestHelper.isSimpleFSIndexInput(expected));
assertTrue(_TestHelper.isSimpleFSIndexInputOpen(expected));
IndexInput one = cr.openInput("f11", IOContext.DEFAULT);
IndexInput one = cr.openInput("f11", newIOContext(random));
assertTrue(isCSIndexInputOpen(one));
IndexInput two = (IndexInput) one.clone();
@ -428,14 +429,14 @@ public class TestCompoundFile extends LuceneTestCase
*/
public void testRandomAccess() throws IOException {
setUp_2();
CompoundFileReader cr = new CompoundFileReader(dir, "f.comp", IOContext.DEFAULT);
CompoundFileReader cr = new CompoundFileReader(dir, "f.comp", newIOContext(random));
// Open two files
IndexInput e1 = dir.openInput("f11", IOContext.DEFAULT);
IndexInput e2 = dir.openInput("f3", IOContext.DEFAULT);
IndexInput e1 = dir.openInput("f11", newIOContext(random));
IndexInput e2 = dir.openInput("f3", newIOContext(random));
IndexInput a1 = cr.openInput("f11", IOContext.DEFAULT);
IndexInput a2 = dir.openInput("f3", IOContext.DEFAULT);
IndexInput a1 = cr.openInput("f11", newIOContext(random));
IndexInput a2 = dir.openInput("f3", newIOContext(random));
// Seek the first pair
e1.seek(100);
@ -507,11 +508,11 @@ public class TestCompoundFile extends LuceneTestCase
*/
public void testRandomAccessClones() throws IOException {
setUp_2();
CompoundFileReader cr = new CompoundFileReader(dir, "f.comp", IOContext.DEFAULT);
CompoundFileReader cr = new CompoundFileReader(dir, "f.comp", newIOContext(random));
// Open two files
IndexInput e1 = cr.openInput("f11", IOContext.DEFAULT);
IndexInput e2 = cr.openInput("f3", IOContext.DEFAULT);
IndexInput e1 = cr.openInput("f11", newIOContext(random));
IndexInput e2 = cr.openInput("f3", newIOContext(random));
IndexInput a1 = (IndexInput) e1.clone();
IndexInput a2 = (IndexInput) e2.clone();
@ -584,11 +585,11 @@ public class TestCompoundFile extends LuceneTestCase
public void testFileNotFound() throws IOException {
setUp_2();
CompoundFileReader cr = new CompoundFileReader(dir, "f.comp", IOContext.DEFAULT);
CompoundFileReader cr = new CompoundFileReader(dir, "f.comp", newIOContext(random));
// Open two files
try {
cr.openInput("bogus", IOContext.DEFAULT);
cr.openInput("bogus", newIOContext(random));
fail("File not found");
} catch (IOException e) {
@ -602,8 +603,8 @@ public class TestCompoundFile extends LuceneTestCase
public void testReadPastEOF() throws IOException {
setUp_2();
CompoundFileReader cr = new CompoundFileReader(dir, "f.comp", IOContext.DEFAULT);
IndexInput is = cr.openInput("f2", IOContext.DEFAULT);
CompoundFileReader cr = new CompoundFileReader(dir, "f.comp", newIOContext(random));
IndexInput is = cr.openInput("f2", newIOContext(random));
is.seek(is.length() - 10);
byte b[] = new byte[100];
is.readBytes(b, 0, 10);
@ -633,7 +634,7 @@ public class TestCompoundFile extends LuceneTestCase
* will correctly increment the file pointer.
*/
public void testLargeWrites() throws IOException {
IndexOutput os = dir.createOutput("testBufferStart.txt", IOContext.DEFAULT);
IndexOutput os = dir.createOutput("testBufferStart.txt", newIOContext(random));
byte[] largeBuf = new byte[2048];
for (int i=0; i<largeBuf.length; i++) {
@ -655,13 +656,13 @@ public class TestCompoundFile extends LuceneTestCase
createSequenceFile(dir, "d1", (byte) 0, 15);
Directory newDir = newDirectory();
CompoundFileWriter csw = new CompoundFileWriter(newDir, "d.csf", IOContext.DEFAULT);
CompoundFileWriter csw = new CompoundFileWriter(newDir, "d.csf", newIOContext(random));
csw.addFile("d1", dir);
csw.close();
CompoundFileReader csr = new CompoundFileReader(newDir, "d.csf", IOContext.DEFAULT);
IndexInput expected = dir.openInput("d1", IOContext.DEFAULT);
IndexInput actual = csr.openInput("d1", IOContext.DEFAULT);
CompoundFileReader csr = new CompoundFileReader(newDir, "d.csf", newIOContext(random));
IndexInput expected = dir.openInput("d1", newIOContext(random));
IndexInput actual = csr.openInput("d1", newIOContext(random));
assertSameStreams("d1", expected, actual);
assertSameSeekBehavior("d1", expected, actual);
expected.close();

View File

@ -32,11 +32,10 @@ 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.store.IOContext;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;
import org.apache.lucene.index.codecs.CodecProvider;
@ -195,11 +194,11 @@ public class TestDoc extends LuceneTestCase {
private SegmentInfo merge(SegmentInfo si1, SegmentInfo si2, String merged, boolean useCompoundFile)
throws Exception {
IOContext context = IOContext.READ;
IOContext context = newIOContext(random);
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());
SegmentMerger merger = new SegmentMerger(si1.dir, IndexWriterConfig.DEFAULT_TERM_INDEX_INTERVAL, merged, null, null, new FieldInfos(), context);
merger.add(r1);
merger.add(r2);
@ -211,7 +210,7 @@ public class TestDoc extends LuceneTestCase {
false, merger.getSegmentCodecs(), fieldInfos);
if (useCompoundFile) {
Collection<String> filesToDelete = merger.createCompoundFile(merged + ".cfs", info, IOContext.DEFAULT);
Collection<String> filesToDelete = merger.createCompoundFile(merged + ".cfs", info, newIOContext(random));
info.setUseCompoundFile(true);
for (final String fileToDelete : filesToDelete)
si1.dir.deleteFile(fileToDelete);
@ -223,7 +222,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, IOContext.READ);
SegmentReader reader = SegmentReader.get(true, si, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random));
for (int i = 0; i < reader.numDocs(); i++)
out.println(reader.document(i));

View File

@ -223,7 +223,7 @@ public class TestDocTermOrds extends LuceneTestCase {
@Override
public PerDocValues docsProducer(SegmentReadState state) throws IOException {
return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId);
return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId, state.context);
}
}

View File

@ -34,8 +34,9 @@ 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.store.IOContext;
import org.apache.lucene.store.IOContext.Context;
import org.apache.lucene.util.AttributeSource;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
@ -69,7 +70,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, IOContext.READ);
SegmentReader reader = SegmentReader.get(true, info, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random));
assertTrue(reader != null);
Document doc = reader.document(0);
assertTrue(doc != null);
@ -130,7 +131,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, IOContext.READ);
SegmentReader reader = SegmentReader.get(true, info, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random));
DocsAndPositionsEnum termPositions = MultiFields.getTermPositionsEnum(reader, MultiFields.getDeletedDocs(reader),
"repeated", new BytesRef("repeated"));
@ -194,7 +195,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, IOContext.READ);
SegmentReader reader = SegmentReader.get(true, info, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random));
DocsAndPositionsEnum termPositions = reader.fields().terms("f1").docsAndPositions(reader.getDeletedDocs(), new BytesRef("a"), null);
assertTrue(termPositions.nextDoc() != termPositions.NO_MORE_DOCS);
@ -238,7 +239,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, IOContext.READ);
SegmentReader reader = SegmentReader.get(true, info, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random));
DocsAndPositionsEnum termPositions = reader.fields().terms("preanalyzed").docsAndPositions(reader.getDeletedDocs(), new BytesRef("term1"), null);
assertTrue(termPositions.nextDoc() != termPositions.NO_MORE_DOCS);

View File

@ -20,7 +20,6 @@ package org.apache.lucene.index;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;
import org.apache.lucene.document.Document;
import org.apache.lucene.index.IOContext.Context;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IndexOutput;
@ -48,7 +47,7 @@ public class TestFieldInfos extends LuceneTestCase {
assertTrue(fieldInfos.size() == DocHelper.all.size()); //this is all b/c we are using the no-arg constructor
IndexOutput output = dir.createOutput(filename, IOContext.DEFAULT);
IndexOutput output = dir.createOutput(filename, newIOContext(random));
assertTrue(output != null);
//Use a RAMOutputStream

View File

@ -35,6 +35,7 @@ import org.apache.lucene.search.FieldCache;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.BufferedIndexInput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.LuceneTestCase;

View File

@ -21,6 +21,7 @@ import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.search.DefaultSimilarity;
import org.apache.lucene.search.Similarity;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.MockDirectoryWrapper;
@ -91,7 +92,7 @@ public class TestIndexFileDeleter extends LuceneTestCase {
// figure out which field number corresponds to
// "content", and then set our expected file names below
// accordingly:
CompoundFileReader cfsReader = new CompoundFileReader(dir, "_2.cfs", IOContext.DEFAULT);
CompoundFileReader cfsReader = new CompoundFileReader(dir, "_2.cfs", newIOContext(random));
FieldInfos fieldInfos = new FieldInfos(cfsReader, "_2.fnm");
int contentFieldIndex = -1;
for (FieldInfo fi : fieldInfos) {
@ -212,8 +213,8 @@ public class TestIndexFileDeleter extends LuceneTestCase {
}
public void copyFile(Directory dir, String src, String dest) throws IOException {
IndexInput in = dir.openInput(src, IOContext.DEFAULT);
IndexOutput out = dir.createOutput(dest, IOContext.DEFAULT);
IndexInput in = dir.openInput(src, newIOContext(random));
IndexOutput out = dir.createOutput(dest, newIOContext(random));
byte[] b = new byte[1024];
long remainder = in.length();
while(remainder > 0) {

View File

@ -99,10 +99,10 @@ public class TestIndexInput extends LuceneTestCase {
// this test checks the raw IndexInput methods as it uses RAMIndexInput which extends IndexInput directly
public void testRawIndexInputRead() throws IOException {
final RAMDirectory dir = new RAMDirectory();
final IndexOutput os = dir.createOutput("foo", IOContext.DEFAULT);
final IndexOutput os = dir.createOutput("foo", newIOContext(random));
os.writeBytes(READ_TEST_BYTES, READ_TEST_BYTES.length);
os.close();
final IndexInput is = dir.openInput("foo", IOContext.DEFAULT);
final IndexInput is = dir.openInput("foo", newIOContext(random));
checkReads(is);
is.close();
dir.close();

View File

@ -84,7 +84,7 @@ public class TestIndexReaderOnDiskFull extends LuceneTestCase {
// Iterate w/ ever increasing free disk space:
while(!done) {
MockDirectoryWrapper dir = new MockDirectoryWrapper(random, new RAMDirectory(startDir, IOContext.DEFAULT));
MockDirectoryWrapper dir = new MockDirectoryWrapper(random, new RAMDirectory(startDir, newIOContext(random)));
// If IndexReader hits disk full, it can write to
// the same files again.

View File

@ -60,6 +60,7 @@ import org.apache.lucene.search.TopDocs;
import org.apache.lucene.search.spans.SpanTermQuery;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.Lock;
import org.apache.lucene.store.LockFactory;
@ -1029,7 +1030,7 @@ public class TestIndexWriter extends LuceneTestCase {
Directory dir = newDirectory();
try {
// Create my own random file:
IndexOutput out = dir.createOutput("myrandomfile", IOContext.DEFAULT);
IndexOutput out = dir.createOutput("myrandomfile", newIOContext(random));
out.writeByte((byte) 42);
out.close();

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