mirror of https://github.com/apache/lucene.git
LUCENE-2814: stop writing shared doc stores
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1050687 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
b9777a3a46
commit
8584e1016f
|
@ -288,7 +288,6 @@ public class IndexTask extends Task {
|
|||
create ? OpenMode.CREATE : OpenMode.APPEND);
|
||||
LogMergePolicy lmp = (LogMergePolicy) conf.getMergePolicy();
|
||||
lmp.setUseCompoundFile(useCompoundIndex);
|
||||
lmp.setUseCompoundDocStore(useCompoundIndex);
|
||||
lmp.setMergeFactor(mergeFactor);
|
||||
IndexWriter writer = new IndexWriter(dir, conf);
|
||||
int totalFiles = 0;
|
||||
|
|
|
@ -138,7 +138,6 @@ public class TestAppendingCodec extends LuceneTestCase {
|
|||
|
||||
cfg.setCodecProvider(new AppendingCodecProvider());
|
||||
((LogMergePolicy)cfg.getMergePolicy()).setUseCompoundFile(false);
|
||||
((LogMergePolicy)cfg.getMergePolicy()).setUseCompoundDocStore(false);
|
||||
IndexWriter writer = new IndexWriter(dir, cfg);
|
||||
Document doc = new Document();
|
||||
doc.add(newField("f", text, Store.YES, Index.ANALYZED, TermVector.WITH_POSITIONS_OFFSETS));
|
||||
|
|
|
@ -251,7 +251,6 @@ public class Syns2Index
|
|||
IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig(
|
||||
Version.LUCENE_CURRENT, ana).setOpenMode(OpenMode.CREATE));
|
||||
((LogMergePolicy) writer.getConfig().getMergePolicy()).setUseCompoundFile(true); // why?
|
||||
((LogMergePolicy) writer.getConfig().getMergePolicy()).setUseCompoundDocStore(true); // why?
|
||||
Iterator<String> i1 = word2Nums.keySet().iterator();
|
||||
while (i1.hasNext()) // for each word
|
||||
{
|
||||
|
|
|
@ -765,7 +765,7 @@ class DirectoryReader extends IndexReader implements Cloneable {
|
|||
// KeepOnlyLastCommitDeleter:
|
||||
IndexFileDeleter deleter = new IndexFileDeleter(directory,
|
||||
deletionPolicy == null ? new KeepOnlyLastCommitDeletionPolicy() : deletionPolicy,
|
||||
segmentInfos, null, null, codecs);
|
||||
segmentInfos, null, codecs);
|
||||
segmentInfos.updateGeneration(deleter.getLastSegmentInfos());
|
||||
segmentInfos.changed();
|
||||
|
||||
|
|
|
@ -23,7 +23,6 @@ import java.util.Collection;
|
|||
abstract class DocConsumer {
|
||||
abstract DocConsumerPerThread addThread(DocumentsWriterThreadState perThread) throws IOException;
|
||||
abstract void flush(final Collection<DocConsumerPerThread> threads, final SegmentWriteState state) throws IOException;
|
||||
abstract void closeDocStore(final SegmentWriteState state) throws IOException;
|
||||
abstract void abort();
|
||||
abstract boolean freeRAM();
|
||||
}
|
||||
|
|
|
@ -29,10 +29,6 @@ abstract class DocFieldConsumer {
|
|||
* segment */
|
||||
abstract void flush(Map<DocFieldConsumerPerThread,Collection<DocFieldConsumerPerField>> threadsAndFields, SegmentWriteState state) throws IOException;
|
||||
|
||||
/** Called when DocumentsWriter decides to close the doc
|
||||
* stores */
|
||||
abstract void closeDocStore(SegmentWriteState state) throws IOException;
|
||||
|
||||
/** Called when an aborting exception is hit */
|
||||
abstract void abort();
|
||||
|
||||
|
|
|
@ -46,12 +46,6 @@ final class DocFieldProcessor extends DocConsumer {
|
|||
fieldsWriter = new StoredFieldsWriter(docWriter, fieldInfos);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void closeDocStore(SegmentWriteState state) throws IOException {
|
||||
consumer.closeDocStore(state);
|
||||
fieldsWriter.closeDocStore(state);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void flush(Collection<DocConsumerPerThread> threads, SegmentWriteState state) throws IOException {
|
||||
|
||||
|
|
|
@ -73,12 +73,6 @@ final class DocInverter extends DocFieldConsumer {
|
|||
endConsumer.flush(endChildThreadsAndFields, state);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void closeDocStore(SegmentWriteState state) throws IOException {
|
||||
consumer.closeDocStore(state);
|
||||
endConsumer.closeDocStore(state);
|
||||
}
|
||||
|
||||
@Override
|
||||
void abort() {
|
||||
consumer.abort();
|
||||
|
|
|
@ -114,12 +114,9 @@ final class DocumentsWriter {
|
|||
Directory directory;
|
||||
|
||||
String segment; // Current segment we are working on
|
||||
private String docStoreSegment; // Current doc-store segment we are writing
|
||||
private int docStoreOffset; // Current starting doc-store offset of current segment
|
||||
|
||||
private int nextDocID; // Next docID to be added
|
||||
private int numDocsInRAM; // # docs buffered in RAM
|
||||
int numDocsInStore; // # docs written to doc stores
|
||||
private int numDocs; // # of docs added, but not yet flushed
|
||||
|
||||
// Max # ThreadState instances; if there are more threads
|
||||
// than this they share ThreadStates
|
||||
|
@ -139,8 +136,6 @@ final class DocumentsWriter {
|
|||
// this, they wait for others to finish first
|
||||
private final int maxThreadStates;
|
||||
|
||||
List<String> newFiles;
|
||||
|
||||
// Deletes for our still-in-RAM (to be flushed next) segment
|
||||
private SegmentDeletes pendingDeletes = new SegmentDeletes();
|
||||
|
||||
|
@ -323,7 +318,7 @@ final class DocumentsWriter {
|
|||
final boolean doFlush = flushControl.waitUpdate(0, queries.length);
|
||||
synchronized(this) {
|
||||
for (Query query : queries) {
|
||||
pendingDeletes.addQuery(query, numDocsInRAM);
|
||||
pendingDeletes.addQuery(query, numDocs);
|
||||
}
|
||||
}
|
||||
return doFlush;
|
||||
|
@ -332,7 +327,7 @@ final class DocumentsWriter {
|
|||
boolean deleteQuery(Query query) {
|
||||
final boolean doFlush = flushControl.waitUpdate(0, 1);
|
||||
synchronized(this) {
|
||||
pendingDeletes.addQuery(query, numDocsInRAM);
|
||||
pendingDeletes.addQuery(query, numDocs);
|
||||
}
|
||||
return doFlush;
|
||||
}
|
||||
|
@ -341,7 +336,7 @@ final class DocumentsWriter {
|
|||
final boolean doFlush = flushControl.waitUpdate(0, terms.length);
|
||||
synchronized(this) {
|
||||
for (Term term : terms) {
|
||||
pendingDeletes.addTerm(term, numDocsInRAM);
|
||||
pendingDeletes.addTerm(term, numDocs);
|
||||
}
|
||||
}
|
||||
return doFlush;
|
||||
|
@ -350,7 +345,7 @@ final class DocumentsWriter {
|
|||
boolean deleteTerm(Term term, boolean skipWait) {
|
||||
final boolean doFlush = flushControl.waitUpdate(0, 1, skipWait);
|
||||
synchronized(this) {
|
||||
pendingDeletes.addTerm(term, numDocsInRAM);
|
||||
pendingDeletes.addTerm(term, numDocs);
|
||||
}
|
||||
return doFlush;
|
||||
}
|
||||
|
@ -370,21 +365,24 @@ final class DocumentsWriter {
|
|||
* here. */
|
||||
synchronized void setInfoStream(PrintStream infoStream) {
|
||||
this.infoStream = infoStream;
|
||||
for(int i=0;i<threadStates.length;i++)
|
||||
for(int i=0;i<threadStates.length;i++) {
|
||||
threadStates[i].docState.infoStream = infoStream;
|
||||
}
|
||||
}
|
||||
|
||||
synchronized void setMaxFieldLength(int maxFieldLength) {
|
||||
this.maxFieldLength = maxFieldLength;
|
||||
for(int i=0;i<threadStates.length;i++)
|
||||
for(int i=0;i<threadStates.length;i++) {
|
||||
threadStates[i].docState.maxFieldLength = maxFieldLength;
|
||||
}
|
||||
}
|
||||
|
||||
synchronized void setSimilarity(Similarity similarity) {
|
||||
this.similarity = similarity;
|
||||
for(int i=0;i<threadStates.length;i++)
|
||||
for(int i=0;i<threadStates.length;i++) {
|
||||
threadStates[i].docState.similarity = similarity;
|
||||
}
|
||||
}
|
||||
|
||||
/** Set how much RAM we can use before flushing. */
|
||||
synchronized void setRAMBufferSizeMB(double mb) {
|
||||
|
@ -424,126 +422,14 @@ final class DocumentsWriter {
|
|||
}
|
||||
|
||||
/** Returns how many docs are currently buffered in RAM. */
|
||||
synchronized int getNumDocsInRAM() {
|
||||
return numDocsInRAM;
|
||||
}
|
||||
|
||||
/** Returns the current doc store segment we are writing
|
||||
* to. */
|
||||
synchronized String getDocStoreSegment() {
|
||||
return docStoreSegment;
|
||||
}
|
||||
|
||||
/** Returns the doc offset into the shared doc store for
|
||||
* the current buffered docs. */
|
||||
synchronized int getDocStoreOffset() {
|
||||
return docStoreOffset;
|
||||
}
|
||||
|
||||
/** Closes the current open doc stores an sets the
|
||||
* docStoreSegment and docStoreUseCFS on the provided
|
||||
* SegmentInfo. */
|
||||
synchronized void closeDocStore(SegmentWriteState flushState, IndexWriter writer, IndexFileDeleter deleter, SegmentInfo newSegment, MergePolicy mergePolicy, SegmentInfos segmentInfos) throws IOException {
|
||||
|
||||
final boolean isSeparate = numDocsInRAM == 0 || !segment.equals(docStoreSegment);
|
||||
|
||||
assert docStoreSegment != null;
|
||||
|
||||
if (infoStream != null) {
|
||||
message("closeDocStore: openFiles=" + openFiles + "; segment=" + docStoreSegment + "; docStoreOffset=" + docStoreOffset + "; numDocsInStore=" + numDocsInStore + "; isSeparate=" + isSeparate);
|
||||
}
|
||||
|
||||
closedFiles.clear();
|
||||
consumer.closeDocStore(flushState);
|
||||
flushState.numDocsInStore = 0;
|
||||
assert 0 == openFiles.size();
|
||||
|
||||
if (isSeparate) {
|
||||
flushState.flushedFiles.clear();
|
||||
|
||||
if (mergePolicy.useCompoundDocStore(segmentInfos)) {
|
||||
|
||||
final String compoundFileName = IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.COMPOUND_FILE_STORE_EXTENSION);
|
||||
|
||||
if (infoStream != null) {
|
||||
message("closeDocStore: create compound file " + compoundFileName);
|
||||
}
|
||||
|
||||
boolean success = false;
|
||||
try {
|
||||
|
||||
CompoundFileWriter cfsWriter = new CompoundFileWriter(directory, compoundFileName);
|
||||
for (final String file : closedFiles) {
|
||||
cfsWriter.addFile(file);
|
||||
}
|
||||
|
||||
// Perform the merge
|
||||
cfsWriter.close();
|
||||
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
deleter.deleteFile(compoundFileName);
|
||||
}
|
||||
}
|
||||
|
||||
// In case the files we just merged into a CFS were
|
||||
// not registered w/ IFD:
|
||||
deleter.deleteNewFiles(closedFiles);
|
||||
|
||||
final int numSegments = segmentInfos.size();
|
||||
for(int i=0;i<numSegments;i++) {
|
||||
SegmentInfo si = segmentInfos.info(i);
|
||||
if (si.getDocStoreOffset() != -1 &&
|
||||
si.getDocStoreSegment().equals(docStoreSegment)) {
|
||||
si.setDocStoreIsCompoundFile(true);
|
||||
}
|
||||
}
|
||||
|
||||
newSegment.setDocStoreIsCompoundFile(true);
|
||||
if (infoStream != null) {
|
||||
message("closeDocStore: after compound file index=" + segmentInfos);
|
||||
}
|
||||
|
||||
writer.checkpoint();
|
||||
}
|
||||
}
|
||||
|
||||
docStoreSegment = null;
|
||||
docStoreOffset = 0;
|
||||
numDocsInStore = 0;
|
||||
}
|
||||
|
||||
private Collection<String> abortedFiles; // List of files that were written before last abort()
|
||||
|
||||
Collection<String> abortedFiles() {
|
||||
return abortedFiles;
|
||||
synchronized int getNumDocs() {
|
||||
return numDocs;
|
||||
}
|
||||
|
||||
void message(String message) {
|
||||
if (infoStream != null)
|
||||
if (infoStream != null) {
|
||||
writer.message("DW: " + message);
|
||||
}
|
||||
|
||||
final List<String> openFiles = new ArrayList<String>();
|
||||
final List<String> closedFiles = new ArrayList<String>();
|
||||
|
||||
/* Returns Collection of files in use by this instance,
|
||||
* including any flushed segments. */
|
||||
@SuppressWarnings("unchecked")
|
||||
synchronized List<String> openFiles() {
|
||||
return (List<String>) ((ArrayList<String>) openFiles).clone();
|
||||
}
|
||||
|
||||
synchronized void addOpenFile(String name) {
|
||||
assert !openFiles.contains(name);
|
||||
openFiles.add(name);
|
||||
}
|
||||
|
||||
synchronized void removeOpenFile(String name) {
|
||||
assert openFiles.contains(name);
|
||||
openFiles.remove(name);
|
||||
closedFiles.add(name);
|
||||
}
|
||||
|
||||
synchronized void setAborting() {
|
||||
|
@ -558,7 +444,6 @@ final class DocumentsWriter {
|
|||
* currently buffered docs. This resets our state,
|
||||
* discarding any docs added since last flush. */
|
||||
synchronized void abort() throws IOException {
|
||||
|
||||
if (infoStream != null) {
|
||||
message("docWriter: abort");
|
||||
}
|
||||
|
@ -582,19 +467,11 @@ final class DocumentsWriter {
|
|||
|
||||
waitQueue.waitingBytes = 0;
|
||||
|
||||
try {
|
||||
abortedFiles = openFiles();
|
||||
} catch (Throwable t) {
|
||||
abortedFiles = null;
|
||||
}
|
||||
|
||||
pendingDeletes.clear();
|
||||
|
||||
openFiles.clear();
|
||||
|
||||
for(int i=0;i<threadStates.length;i++)
|
||||
for (DocumentsWriterThreadState threadState : threadStates)
|
||||
try {
|
||||
threadStates[i].consumer.abort();
|
||||
threadState.consumer.abort();
|
||||
} catch (Throwable t) {
|
||||
}
|
||||
|
||||
|
@ -603,10 +480,6 @@ final class DocumentsWriter {
|
|||
} catch (Throwable t) {
|
||||
}
|
||||
|
||||
docStoreSegment = null;
|
||||
numDocsInStore = 0;
|
||||
docStoreOffset = 0;
|
||||
|
||||
// Reset all postings data
|
||||
doAfterFlush();
|
||||
success = true;
|
||||
|
@ -614,7 +487,7 @@ final class DocumentsWriter {
|
|||
aborting = false;
|
||||
notifyAll();
|
||||
if (infoStream != null) {
|
||||
message("docWriter: done abort; abortedFiles=" + abortedFiles + " success=" + success);
|
||||
message("docWriter: done abort; success=" + success);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -626,22 +499,25 @@ final class DocumentsWriter {
|
|||
threadBindings.clear();
|
||||
waitQueue.reset();
|
||||
segment = null;
|
||||
numDocsInRAM = 0;
|
||||
numDocs = 0;
|
||||
nextDocID = 0;
|
||||
bufferIsFull = false;
|
||||
for(int i=0;i<threadStates.length;i++)
|
||||
for(int i=0;i<threadStates.length;i++) {
|
||||
threadStates[i].doAfterFlush();
|
||||
}
|
||||
}
|
||||
|
||||
private synchronized boolean allThreadsIdle() {
|
||||
for(int i=0;i<threadStates.length;i++)
|
||||
if (!threadStates[i].isIdle)
|
||||
for(int i=0;i<threadStates.length;i++) {
|
||||
if (!threadStates[i].isIdle) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
synchronized boolean anyChanges() {
|
||||
return numDocsInRAM != 0 || pendingDeletes.any();
|
||||
return numDocs != 0 || pendingDeletes.any();
|
||||
}
|
||||
|
||||
// for testing
|
||||
|
@ -680,14 +556,14 @@ final class DocumentsWriter {
|
|||
|
||||
/** Flush all pending docs to a new segment */
|
||||
// Lock order: IW -> DW
|
||||
synchronized SegmentInfo flush(IndexWriter writer, boolean closeDocStore, IndexFileDeleter deleter, MergePolicy mergePolicy, SegmentInfos segmentInfos) throws IOException {
|
||||
synchronized SegmentInfo flush(IndexWriter writer, IndexFileDeleter deleter, MergePolicy mergePolicy, SegmentInfos segmentInfos) throws IOException {
|
||||
|
||||
// We change writer's segmentInfos:
|
||||
assert Thread.holdsLock(writer);
|
||||
|
||||
waitIdle();
|
||||
|
||||
if (numDocsInRAM == 0 && numDocsInStore == 0) {
|
||||
if (numDocs == 0) {
|
||||
// nothing to do!
|
||||
if (infoStream != null) {
|
||||
message("flush: no docs; skipping");
|
||||
|
@ -709,68 +585,37 @@ final class DocumentsWriter {
|
|||
SegmentInfo newSegment;
|
||||
|
||||
try {
|
||||
|
||||
assert waitQueue.waitingBytes == 0;
|
||||
|
||||
assert docStoreSegment != null || numDocsInRAM == 0: "dss=" + docStoreSegment + " numDocsInRAM=" + numDocsInRAM;
|
||||
|
||||
assert numDocsInStore >= numDocsInRAM: "numDocsInStore=" + numDocsInStore + " numDocsInRAM=" + numDocsInRAM;
|
||||
|
||||
final SegmentWriteState flushState = new SegmentWriteState(infoStream, directory, segment, fieldInfos,
|
||||
docStoreSegment, numDocsInRAM, numDocsInStore, writer.getConfig().getTermIndexInterval(),
|
||||
SegmentCodecs.build(fieldInfos, writer.codecs));
|
||||
|
||||
newSegment = new SegmentInfo(segment, numDocsInRAM, directory, false, -1, null, false, hasProx(), flushState.segmentCodecs, false);
|
||||
|
||||
if (!closeDocStore || docStoreOffset != 0) {
|
||||
newSegment.setDocStoreSegment(docStoreSegment);
|
||||
newSegment.setDocStoreOffset(docStoreOffset);
|
||||
}
|
||||
|
||||
if (closeDocStore) {
|
||||
closeDocStore(flushState, writer, deleter, newSegment, mergePolicy, segmentInfos);
|
||||
}
|
||||
|
||||
boolean hasVectors = flushState.hasVectors;
|
||||
|
||||
if (numDocsInRAM > 0) {
|
||||
|
||||
assert nextDocID == numDocsInRAM;
|
||||
assert nextDocID == numDocs;
|
||||
assert waitQueue.numWaiting == 0;
|
||||
assert waitQueue.waitingBytes == 0;
|
||||
|
||||
if (infoStream != null) {
|
||||
message("flush postings as segment " + segment + " numDocs=" + numDocsInRAM);
|
||||
message("flush postings as segment " + segment + " numDocs=" + numDocs);
|
||||
}
|
||||
|
||||
final Collection<DocConsumerPerThread> threads = new HashSet<DocConsumerPerThread>();
|
||||
for(int i=0;i<threadStates.length;i++) {
|
||||
threads.add(threadStates[i].consumer);
|
||||
final SegmentWriteState flushState = new SegmentWriteState(infoStream, directory, segment, fieldInfos,
|
||||
numDocs, writer.getConfig().getTermIndexInterval(),
|
||||
SegmentCodecs.build(fieldInfos, writer.codecs));
|
||||
|
||||
newSegment = new SegmentInfo(segment, numDocs, directory, false, hasProx(), flushState.segmentCodecs, false);
|
||||
|
||||
Collection<DocConsumerPerThread> threads = new HashSet<DocConsumerPerThread>();
|
||||
for (DocumentsWriterThreadState threadState : threadStates) {
|
||||
threads.add(threadState.consumer);
|
||||
}
|
||||
|
||||
final long startNumBytesUsed = bytesUsed();
|
||||
long startNumBytesUsed = bytesUsed();
|
||||
|
||||
consumer.flush(threads, flushState);
|
||||
|
||||
hasVectors |= flushState.hasVectors;
|
||||
|
||||
if (hasVectors) {
|
||||
if (infoStream != null) {
|
||||
message("new segment has vectors");
|
||||
}
|
||||
newSegment.setHasVectors(true);
|
||||
} else {
|
||||
if (infoStream != null) {
|
||||
message("new segment has no vectors");
|
||||
}
|
||||
}
|
||||
newSegment.setHasVectors(flushState.hasVectors);
|
||||
|
||||
if (infoStream != null) {
|
||||
message("new segment has " + (flushState.hasVectors ? "vectors" : "no vectors"));
|
||||
message("flushedFiles=" + flushState.flushedFiles);
|
||||
message("flushed codecs=" + newSegment.getSegmentCodecs());
|
||||
}
|
||||
|
||||
if (mergePolicy.useCompoundFile(segmentInfos, newSegment)) {
|
||||
|
||||
final String cfsFileName = IndexFileNames.segmentFileName(segment, "", IndexFileNames.COMPOUND_FILE_EXTENSION);
|
||||
|
||||
if (infoStream != null) {
|
||||
|
@ -790,18 +635,10 @@ final class DocumentsWriter {
|
|||
if (infoStream != null) {
|
||||
message("flush: segment=" + newSegment);
|
||||
final long newSegmentSize = newSegment.sizeInBytes();
|
||||
String message = " ramUsed=" + nf.format(startNumBytesUsed/1024./1024.) + " MB" +
|
||||
message(" ramUsed=" + nf.format(startNumBytesUsed / 1024. / 1024.) + " MB" +
|
||||
" newFlushedSize=" + nf.format(newSegmentSize / 1024 / 1024) + " MB" +
|
||||
" docs/MB=" + nf.format(numDocsInRAM/(newSegmentSize/1024./1024.)) +
|
||||
" new/old=" + nf.format(100.0*newSegmentSize/startNumBytesUsed) + "%";
|
||||
message(message);
|
||||
}
|
||||
|
||||
} else {
|
||||
if (infoStream != null) {
|
||||
message("skip flushing segment: no docs");
|
||||
}
|
||||
newSegment = null;
|
||||
" docs/MB=" + nf.format(numDocs / (newSegmentSize / 1024. / 1024.)) +
|
||||
" new/old=" + nf.format(100.0 * newSegmentSize / startNumBytesUsed) + "%");
|
||||
}
|
||||
|
||||
success = true;
|
||||
|
@ -820,8 +657,6 @@ final class DocumentsWriter {
|
|||
// Lock order: IW -> DW -> BD
|
||||
pushDeletes(newSegment, segmentInfos);
|
||||
|
||||
docStoreOffset = numDocsInStore;
|
||||
|
||||
return newSegment;
|
||||
}
|
||||
|
||||
|
@ -830,17 +665,6 @@ final class DocumentsWriter {
|
|||
notifyAll();
|
||||
}
|
||||
|
||||
synchronized void initSegmentName(boolean onlyDocStore) {
|
||||
if (segment == null && (!onlyDocStore || docStoreSegment == null)) {
|
||||
segment = writer.newSegmentName();
|
||||
assert numDocsInRAM == 0;
|
||||
}
|
||||
if (docStoreSegment == null) {
|
||||
docStoreSegment = segment;
|
||||
assert numDocsInStore == 0;
|
||||
}
|
||||
}
|
||||
|
||||
/** Returns a free (idle) ThreadState that may be used for
|
||||
* indexing this one document. This call also pauses if a
|
||||
* flush is pending. If delTerm is non-null then we
|
||||
|
@ -862,17 +686,19 @@ final class DocumentsWriter {
|
|||
DocumentsWriterThreadState minThreadState = null;
|
||||
for(int i=0;i<threadStates.length;i++) {
|
||||
DocumentsWriterThreadState ts = threadStates[i];
|
||||
if (minThreadState == null || ts.numThreads < minThreadState.numThreads)
|
||||
if (minThreadState == null || ts.numThreads < minThreadState.numThreads) {
|
||||
minThreadState = ts;
|
||||
}
|
||||
}
|
||||
if (minThreadState != null && (minThreadState.numThreads == 0 || threadStates.length >= maxThreadStates)) {
|
||||
state = minThreadState;
|
||||
state.numThreads++;
|
||||
} else {
|
||||
// Just create a new "private" thread state
|
||||
DocumentsWriterThreadState[] newArray = new DocumentsWriterThreadState[1+threadStates.length];
|
||||
if (threadStates.length > 0)
|
||||
if (threadStates.length > 0) {
|
||||
System.arraycopy(threadStates, 0, newArray, 0, threadStates.length);
|
||||
}
|
||||
state = newArray[threadStates.length] = new DocumentsWriterThreadState(this);
|
||||
threadStates = newArray;
|
||||
}
|
||||
|
@ -886,7 +712,10 @@ final class DocumentsWriter {
|
|||
|
||||
// Allocate segment name if this is the first doc since
|
||||
// last flush:
|
||||
initSegmentName(false);
|
||||
if (segment == null) {
|
||||
segment = writer.newSegmentName();
|
||||
assert numDocs == 0;
|
||||
}
|
||||
|
||||
state.docState.docID = nextDocID++;
|
||||
|
||||
|
@ -894,7 +723,7 @@ final class DocumentsWriter {
|
|||
pendingDeletes.addTerm(delTerm, state.docState.docID);
|
||||
}
|
||||
|
||||
numDocsInRAM++;
|
||||
numDocs++;
|
||||
state.isIdle = false;
|
||||
return state;
|
||||
}
|
||||
|
@ -1038,15 +867,16 @@ final class DocumentsWriter {
|
|||
|
||||
final boolean doPause;
|
||||
|
||||
if (docWriter != null)
|
||||
if (docWriter != null) {
|
||||
doPause = waitQueue.add(docWriter);
|
||||
else {
|
||||
} else {
|
||||
skipDocWriter.docID = perThread.docState.docID;
|
||||
doPause = waitQueue.add(skipDocWriter);
|
||||
}
|
||||
|
||||
if (doPause)
|
||||
if (doPause) {
|
||||
waitForWaitQueue();
|
||||
}
|
||||
|
||||
perThread.isIdle = true;
|
||||
|
||||
|
@ -1095,7 +925,7 @@ final class DocumentsWriter {
|
|||
final static int INT_BLOCK_SIZE = 1 << INT_BLOCK_SHIFT;
|
||||
final static int INT_BLOCK_MASK = INT_BLOCK_SIZE - 1;
|
||||
|
||||
private ArrayList<int[]> freeIntBlocks = new ArrayList<int[]>();
|
||||
private List<int[]> freeIntBlocks = new ArrayList<int[]>();
|
||||
|
||||
/* Allocate another int[] from the shared pool */
|
||||
synchronized int[] getIntBlock() {
|
||||
|
@ -1104,8 +934,9 @@ final class DocumentsWriter {
|
|||
if (0 == size) {
|
||||
b = new int[INT_BLOCK_SIZE];
|
||||
bytesUsed.addAndGet(INT_BLOCK_SIZE*RamUsageEstimator.NUM_BYTES_INT);
|
||||
} else
|
||||
} else {
|
||||
b = freeIntBlocks.remove(size-1);
|
||||
}
|
||||
return b;
|
||||
}
|
||||
|
||||
|
@ -1158,12 +989,13 @@ final class DocumentsWriter {
|
|||
|
||||
if (doBalance) {
|
||||
|
||||
if (infoStream != null)
|
||||
if (infoStream != null) {
|
||||
message(" RAM: balance allocations: usedMB=" + toMB(bytesUsed()) +
|
||||
" vs trigger=" + toMB(ramBufferSize) +
|
||||
" deletesMB=" + toMB(deletesRAMUsed) +
|
||||
" byteBlockFree=" + toMB(byteBlockAllocator.bytesUsed()) +
|
||||
" perDocFree=" + toMB(perDocAllocator.bytesUsed()));
|
||||
}
|
||||
|
||||
final long startBytesUsed = bytesUsed() + deletesRAMUsed;
|
||||
|
||||
|
@ -1184,11 +1016,12 @@ final class DocumentsWriter {
|
|||
// Nothing else to free -- must flush now.
|
||||
bufferIsFull = bytesUsed()+deletesRAMUsed > ramBufferSize;
|
||||
if (infoStream != null) {
|
||||
if (bytesUsed()+deletesRAMUsed > ramBufferSize)
|
||||
if (bytesUsed()+deletesRAMUsed > ramBufferSize) {
|
||||
message(" nothing to free; set bufferIsFull");
|
||||
else
|
||||
} else {
|
||||
message(" nothing to free");
|
||||
}
|
||||
}
|
||||
break;
|
||||
}
|
||||
|
||||
|
@ -1204,17 +1037,19 @@ final class DocumentsWriter {
|
|||
}
|
||||
}
|
||||
|
||||
if ((3 == iter % 4) && any)
|
||||
if ((3 == iter % 4) && any) {
|
||||
// Ask consumer to free any recycled state
|
||||
any = consumer.freeRAM();
|
||||
}
|
||||
|
||||
iter++;
|
||||
}
|
||||
|
||||
if (infoStream != null)
|
||||
if (infoStream != null) {
|
||||
message(" after free: freedMB=" + nf.format((startBytesUsed-bytesUsed()-deletesRAMUsed)/1024./1024.) + " usedMB=" + nf.format((bytesUsed()+deletesRAMUsed)/1024./1024.));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
final WaitQueue waitQueue = new WaitQueue();
|
||||
|
||||
|
@ -1265,11 +1100,11 @@ final class DocumentsWriter {
|
|||
try {
|
||||
doc.finish();
|
||||
nextWriteDocID++;
|
||||
numDocsInStore++;
|
||||
nextWriteLoc++;
|
||||
assert nextWriteLoc <= waiting.length;
|
||||
if (nextWriteLoc == waiting.length)
|
||||
if (nextWriteLoc == waiting.length) {
|
||||
nextWriteLoc = 0;
|
||||
}
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
|
@ -1316,8 +1151,9 @@ final class DocumentsWriter {
|
|||
}
|
||||
|
||||
int loc = nextWriteLoc + gap;
|
||||
if (loc >= waiting.length)
|
||||
if (loc >= waiting.length) {
|
||||
loc -= waiting.length;
|
||||
}
|
||||
|
||||
// We should only wrap one time
|
||||
assert loc < waiting.length;
|
||||
|
|
|
@ -25,9 +25,9 @@ import org.apache.lucene.store.Directory;
|
|||
import org.apache.lucene.store.RAMOutputStream;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
final class FieldsWriter
|
||||
{
|
||||
final class FieldsWriter {
|
||||
static final byte FIELD_IS_TOKENIZED = 0x1;
|
||||
static final byte FIELD_IS_BINARY = 0x2;
|
||||
|
||||
|
@ -42,71 +42,40 @@ final class FieldsWriter
|
|||
// when removing support for old versions, leave the last supported version here
|
||||
static final int FORMAT_MINIMUM = FORMAT_LUCENE_3_0_NO_COMPRESSED_FIELDS;
|
||||
|
||||
// If null - we were supplied with streams, if notnull - we manage them ourselves
|
||||
private Directory directory;
|
||||
private String segment;
|
||||
private FieldInfos fieldInfos;
|
||||
|
||||
private IndexOutput fieldsStream;
|
||||
|
||||
private IndexOutput indexStream;
|
||||
|
||||
private boolean doClose;
|
||||
|
||||
FieldsWriter(Directory d, String segment, FieldInfos fn) throws IOException {
|
||||
FieldsWriter(Directory directory, String segment, FieldInfos fn) throws IOException {
|
||||
this.directory = directory;
|
||||
this.segment = segment;
|
||||
fieldInfos = fn;
|
||||
|
||||
boolean success = false;
|
||||
final String fieldsName = IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELDS_EXTENSION);
|
||||
try {
|
||||
fieldsStream = d.createOutput(fieldsName);
|
||||
fieldsStream = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELDS_EXTENSION));
|
||||
indexStream = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELDS_INDEX_EXTENSION));
|
||||
|
||||
fieldsStream.writeInt(FORMAT_CURRENT);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
try {
|
||||
close();
|
||||
} catch (Throwable t) {
|
||||
// Suppress so we keep throwing the original exception
|
||||
}
|
||||
try {
|
||||
d.deleteFile(fieldsName);
|
||||
} catch (Throwable t) {
|
||||
// Suppress so we keep throwing the original exception
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
success = false;
|
||||
final String indexName = IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELDS_INDEX_EXTENSION);
|
||||
try {
|
||||
indexStream = d.createOutput(indexName);
|
||||
indexStream.writeInt(FORMAT_CURRENT);
|
||||
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
try {
|
||||
close();
|
||||
} catch (IOException ioe) {
|
||||
}
|
||||
try {
|
||||
d.deleteFile(fieldsName);
|
||||
} catch (Throwable t) {
|
||||
// Suppress so we keep throwing the original exception
|
||||
}
|
||||
try {
|
||||
d.deleteFile(indexName);
|
||||
} catch (Throwable t) {
|
||||
// Suppress so we keep throwing the original exception
|
||||
abort();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
doClose = true;
|
||||
}
|
||||
|
||||
FieldsWriter(IndexOutput fdx, IndexOutput fdt, FieldInfos fn) {
|
||||
directory = null;
|
||||
segment = null;
|
||||
fieldInfos = fn;
|
||||
fieldsStream = fdt;
|
||||
indexStream = fdx;
|
||||
doClose = false;
|
||||
}
|
||||
|
||||
void setFieldsStream(IndexOutput stream) {
|
||||
|
@ -128,42 +97,29 @@ final class FieldsWriter
|
|||
fieldsStream.writeVInt(0);
|
||||
}
|
||||
|
||||
void flush() throws IOException {
|
||||
indexStream.flush();
|
||||
fieldsStream.flush();
|
||||
void close() throws IOException {
|
||||
if (directory != null) {
|
||||
try {
|
||||
IOUtils.closeSafely(fieldsStream, indexStream);
|
||||
} finally {
|
||||
fieldsStream = indexStream = null;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
final void close() throws IOException {
|
||||
if (doClose) {
|
||||
void abort() {
|
||||
if (directory != null) {
|
||||
try {
|
||||
if (fieldsStream != null) {
|
||||
close();
|
||||
} catch (IOException ignored) {
|
||||
}
|
||||
try {
|
||||
fieldsStream.close();
|
||||
} finally {
|
||||
fieldsStream = null;
|
||||
directory.deleteFile(IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELDS_EXTENSION));
|
||||
} catch (IOException ignored) {
|
||||
}
|
||||
}
|
||||
} catch (IOException ioe) {
|
||||
try {
|
||||
if (indexStream != null) {
|
||||
try {
|
||||
indexStream.close();
|
||||
} finally {
|
||||
indexStream = null;
|
||||
}
|
||||
}
|
||||
} catch (IOException ioe2) {
|
||||
// Ignore so we throw only first IOException hit
|
||||
}
|
||||
throw ioe;
|
||||
} finally {
|
||||
if (indexStream != null) {
|
||||
try {
|
||||
indexStream.close();
|
||||
} finally {
|
||||
indexStream = null;
|
||||
}
|
||||
}
|
||||
directory.deleteFile(IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELDS_INDEX_EXTENSION));
|
||||
} catch (IOException ignored) {
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,7 +20,6 @@ package org.apache.lucene.index;
|
|||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Comparator;
|
||||
|
@ -38,9 +37,6 @@ final class FreqProxTermsWriter extends TermsHashConsumer {
|
|||
return new FreqProxTermsWriterPerThread(perThread);
|
||||
}
|
||||
|
||||
@Override
|
||||
void closeDocStore(SegmentWriteState state) {}
|
||||
|
||||
@Override
|
||||
void abort() {}
|
||||
|
||||
|
|
|
@ -21,14 +21,7 @@ import java.io.FileNotFoundException;
|
|||
import java.io.FilenameFilter;
|
||||
import java.io.IOException;
|
||||
import java.io.PrintStream;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Date;
|
||||
import java.util.HashMap;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.*;
|
||||
|
||||
import org.apache.lucene.index.codecs.CodecProvider;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
@ -101,7 +94,6 @@ final class IndexFileDeleter {
|
|||
private PrintStream infoStream;
|
||||
private Directory directory;
|
||||
private IndexDeletionPolicy policy;
|
||||
private DocumentsWriter docWriter;
|
||||
|
||||
final boolean startingCommitDeleted;
|
||||
private SegmentInfos lastSegmentInfos;
|
||||
|
@ -112,9 +104,10 @@ final class IndexFileDeleter {
|
|||
|
||||
void setInfoStream(PrintStream infoStream) {
|
||||
this.infoStream = infoStream;
|
||||
if (infoStream != null)
|
||||
if (infoStream != null) {
|
||||
message("setInfoStream deletionPolicy=" + policy);
|
||||
}
|
||||
}
|
||||
|
||||
private void message(String message) {
|
||||
infoStream.println("IFD [" + new Date() + "; " + Thread.currentThread().getName() + "]: " + message);
|
||||
|
@ -130,17 +123,14 @@ final class IndexFileDeleter {
|
|||
* @throws CorruptIndexException if the index is corrupt
|
||||
* @throws IOException if there is a low-level IO error
|
||||
*/
|
||||
public IndexFileDeleter(Directory directory, IndexDeletionPolicy policy, SegmentInfos segmentInfos, PrintStream infoStream, DocumentsWriter docWriter,
|
||||
CodecProvider codecs)
|
||||
throws CorruptIndexException, IOException {
|
||||
|
||||
this.docWriter = docWriter;
|
||||
public IndexFileDeleter(Directory directory, IndexDeletionPolicy policy, SegmentInfos segmentInfos, PrintStream infoStream, CodecProvider codecs) throws CorruptIndexException, IOException {
|
||||
this.infoStream = infoStream;
|
||||
|
||||
final String currentSegmentsFile = segmentInfos.getCurrentSegmentFileName();
|
||||
|
||||
if (infoStream != null)
|
||||
if (infoStream != null) {
|
||||
message("init: current segments file is \"" + currentSegmentsFile + "\"; deletionPolicy=" + policy);
|
||||
}
|
||||
|
||||
this.policy = policy;
|
||||
this.directory = directory;
|
||||
|
@ -229,8 +219,9 @@ final class IndexFileDeleter {
|
|||
} catch (IOException e) {
|
||||
throw new CorruptIndexException("failed to locate current segments_N file");
|
||||
}
|
||||
if (infoStream != null)
|
||||
if (infoStream != null) {
|
||||
message("forced open of current segments file " + segmentInfos.getCurrentSegmentFileName());
|
||||
}
|
||||
currentCommitPoint = new CommitPoint(commitsToDelete, directory, sis);
|
||||
commits.add(currentCommitPoint);
|
||||
incRef(sis, true);
|
||||
|
@ -360,8 +351,9 @@ final class IndexFileDeleter {
|
|||
// DecRef old files from the last checkpoint, if any:
|
||||
int size = lastFiles.size();
|
||||
if (size > 0) {
|
||||
for(int i=0;i<size;i++)
|
||||
for(int i=0;i<size;i++) {
|
||||
decRef(lastFiles.get(i));
|
||||
}
|
||||
lastFiles.clear();
|
||||
}
|
||||
|
||||
|
@ -394,8 +386,9 @@ final class IndexFileDeleter {
|
|||
deletable = null;
|
||||
int size = oldDeletable.size();
|
||||
for(int i=0;i<size;i++) {
|
||||
if (infoStream != null)
|
||||
if (infoStream != null) {
|
||||
message("delete pending file " + oldDeletable.get(i));
|
||||
}
|
||||
deleteFile(oldDeletable.get(i));
|
||||
}
|
||||
}
|
||||
|
@ -444,31 +437,14 @@ final class IndexFileDeleter {
|
|||
// Decref files for commits that were deleted by the policy:
|
||||
deleteCommits();
|
||||
} else {
|
||||
|
||||
final List<String> docWriterFiles;
|
||||
if (docWriter != null) {
|
||||
docWriterFiles = docWriter.openFiles();
|
||||
if (docWriterFiles != null)
|
||||
// We must incRef these files before decRef'ing
|
||||
// last files to make sure we don't accidentally
|
||||
// delete them:
|
||||
incRef(docWriterFiles);
|
||||
} else
|
||||
docWriterFiles = null;
|
||||
|
||||
// DecRef old files from the last checkpoint, if any:
|
||||
int size = lastFiles.size();
|
||||
if (size > 0) {
|
||||
for(int i=0;i<size;i++)
|
||||
decRef(lastFiles.get(i));
|
||||
lastFiles.clear();
|
||||
for (Collection<String> lastFile : lastFiles) {
|
||||
decRef(lastFile);
|
||||
}
|
||||
lastFiles.clear();
|
||||
|
||||
// Save files so we can decr on next checkpoint/commit:
|
||||
lastFiles.add(segmentInfos.files(directory, false));
|
||||
|
||||
if (docWriterFiles != null)
|
||||
lastFiles.add(docWriterFiles);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -539,9 +515,10 @@ final class IndexFileDeleter {
|
|||
}
|
||||
|
||||
void deleteFiles(List<String> files) throws IOException {
|
||||
for(final String file: files)
|
||||
for(final String file: files) {
|
||||
deleteFile(file);
|
||||
}
|
||||
}
|
||||
|
||||
/** Deletes the specified files, but only if they are new
|
||||
* (have not yet been incref'd). */
|
||||
|
@ -699,6 +676,5 @@ final class IndexFileDeleter {
|
|||
public boolean isDeleted() {
|
||||
return deleted;
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
@ -346,7 +346,7 @@ public class IndexWriter implements Closeable {
|
|||
// just like we do when loading segments_N
|
||||
IndexReader r;
|
||||
synchronized(this) {
|
||||
flush(false, true, true);
|
||||
flush(false, true);
|
||||
r = new DirectoryReader(this, segmentInfos, config.getReaderTermsIndexDivisor(), codecs);
|
||||
if (infoStream != null) {
|
||||
message("return reader version=" + r.getVersion() + " reader=" + r);
|
||||
|
@ -773,7 +773,7 @@ public class IndexWriter implements Closeable {
|
|||
// KeepOnlyLastCommitDeleter:
|
||||
deleter = new IndexFileDeleter(directory,
|
||||
conf.getIndexDeletionPolicy(),
|
||||
segmentInfos, infoStream, docWriter, codecs);
|
||||
segmentInfos, infoStream, codecs);
|
||||
|
||||
if (deleter.startingCommitDeleted) {
|
||||
// Deletion policy deleted the "head" commit point.
|
||||
|
@ -1037,7 +1037,7 @@ public class IndexWriter implements Closeable {
|
|||
// Only allow a new merge to be triggered if we are
|
||||
// going to wait for merges:
|
||||
if (!hitOOM) {
|
||||
flush(waitForMerges, true, true);
|
||||
flush(waitForMerges, true);
|
||||
}
|
||||
|
||||
if (waitForMerges)
|
||||
|
@ -1111,7 +1111,7 @@ public class IndexWriter implements Closeable {
|
|||
public synchronized int maxDoc() {
|
||||
int count;
|
||||
if (docWriter != null)
|
||||
count = docWriter.getNumDocsInRAM();
|
||||
count = docWriter.getNumDocs();
|
||||
else
|
||||
count = 0;
|
||||
|
||||
|
@ -1129,7 +1129,7 @@ public class IndexWriter implements Closeable {
|
|||
public synchronized int numDocs() throws IOException {
|
||||
int count;
|
||||
if (docWriter != null)
|
||||
count = docWriter.getNumDocsInRAM();
|
||||
count = docWriter.getNumDocs();
|
||||
else
|
||||
count = 0;
|
||||
|
||||
|
@ -1244,27 +1244,11 @@ public class IndexWriter implements Closeable {
|
|||
doFlush = docWriter.updateDocument(doc, analyzer, null);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
|
||||
if (infoStream != null) {
|
||||
if (!success && infoStream != null)
|
||||
message("hit exception adding document");
|
||||
}
|
||||
|
||||
synchronized (this) {
|
||||
// If docWriter has some aborted files that were
|
||||
// never incref'd, then we clean them up here
|
||||
deleter.checkpoint(segmentInfos, false);
|
||||
if (docWriter != null) {
|
||||
final Collection<String> files = docWriter.abortedFiles();
|
||||
if (files != null) {
|
||||
deleter.deleteNewFiles(files);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
if (doFlush)
|
||||
flush(true, false, false);
|
||||
flush(true, false);
|
||||
} catch (OutOfMemoryError oom) {
|
||||
handleOOM(oom, "addDocument");
|
||||
}
|
||||
|
@ -1285,7 +1269,7 @@ public class IndexWriter implements Closeable {
|
|||
ensureOpen();
|
||||
try {
|
||||
if (docWriter.deleteTerm(term, false)) {
|
||||
flush(true, false, false);
|
||||
flush(true, false);
|
||||
}
|
||||
} catch (OutOfMemoryError oom) {
|
||||
handleOOM(oom, "deleteDocuments(Term)");
|
||||
|
@ -1309,7 +1293,7 @@ public class IndexWriter implements Closeable {
|
|||
ensureOpen();
|
||||
try {
|
||||
if (docWriter.deleteTerms(terms)) {
|
||||
flush(true, false, false);
|
||||
flush(true, false);
|
||||
}
|
||||
} catch (OutOfMemoryError oom) {
|
||||
handleOOM(oom, "deleteDocuments(Term..)");
|
||||
|
@ -1331,7 +1315,7 @@ public class IndexWriter implements Closeable {
|
|||
ensureOpen();
|
||||
try {
|
||||
if (docWriter.deleteQuery(query)) {
|
||||
flush(true, false, false);
|
||||
flush(true, false);
|
||||
}
|
||||
} catch (OutOfMemoryError oom) {
|
||||
handleOOM(oom, "deleteDocuments(Query)");
|
||||
|
@ -1355,7 +1339,7 @@ public class IndexWriter implements Closeable {
|
|||
ensureOpen();
|
||||
try {
|
||||
if (docWriter.deleteQueries(queries)) {
|
||||
flush(true, false, false);
|
||||
flush(true, false);
|
||||
}
|
||||
} catch (OutOfMemoryError oom) {
|
||||
handleOOM(oom, "deleteDocuments(Query..)");
|
||||
|
@ -1412,26 +1396,11 @@ public class IndexWriter implements Closeable {
|
|||
doFlush = docWriter.updateDocument(doc, analyzer, term);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
|
||||
if (infoStream != null) {
|
||||
if (!success && infoStream != null)
|
||||
message("hit exception updating document");
|
||||
}
|
||||
|
||||
synchronized (this) {
|
||||
// If docWriter has some aborted files that were
|
||||
// never incref'd, then we clean them up here
|
||||
if (docWriter != null) {
|
||||
final Collection<String> files = docWriter.abortedFiles();
|
||||
if (files != null) {
|
||||
deleter.deleteNewFiles(files);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
if (doFlush) {
|
||||
flush(true, false, false);
|
||||
flush(true, false);
|
||||
}
|
||||
} catch (OutOfMemoryError oom) {
|
||||
handleOOM(oom, "updateDocument");
|
||||
|
@ -1445,7 +1414,7 @@ public class IndexWriter implements Closeable {
|
|||
|
||||
// for test purpose
|
||||
final synchronized int getNumBufferedDocuments(){
|
||||
return docWriter.getNumDocsInRAM();
|
||||
return docWriter.getNumDocs();
|
||||
}
|
||||
|
||||
// for test purpose
|
||||
|
@ -1601,7 +1570,7 @@ public class IndexWriter implements Closeable {
|
|||
message("now flush at optimize");
|
||||
}
|
||||
|
||||
flush(true, false, true);
|
||||
flush(true, true);
|
||||
|
||||
synchronized(this) {
|
||||
resetMergeExceptions();
|
||||
|
@ -2141,7 +2110,7 @@ public class IndexWriter implements Closeable {
|
|||
try {
|
||||
if (infoStream != null)
|
||||
message("flush at addIndexes(Directory...)");
|
||||
flush(false, false, true);
|
||||
flush(false, true);
|
||||
|
||||
int docCount = 0;
|
||||
List<SegmentInfo> infos = new ArrayList<SegmentInfo>();
|
||||
|
@ -2252,7 +2221,7 @@ public class IndexWriter implements Closeable {
|
|||
int docCount = merger.merge(); // merge 'em
|
||||
|
||||
SegmentInfo info = new SegmentInfo(mergedName, docCount, directory,
|
||||
false, -1, null, false, merger.fieldInfos().hasProx(), merger.getSegmentCodecs(),
|
||||
false, merger.fieldInfos().hasProx(), merger.getSegmentCodecs(),
|
||||
merger.fieldInfos().hasVectors());
|
||||
setDiagnostics(info, "addIndexes(IndexReader...)");
|
||||
|
||||
|
@ -2348,7 +2317,7 @@ public class IndexWriter implements Closeable {
|
|||
if (infoStream != null)
|
||||
message("prepareCommit: flush");
|
||||
|
||||
flush(true, true, true);
|
||||
flush(true, true);
|
||||
|
||||
startCommit(commitUserData);
|
||||
}
|
||||
|
@ -2465,12 +2434,9 @@ public class IndexWriter implements Closeable {
|
|||
* to the Directory.
|
||||
* @param triggerMerge if true, we may merge segments (if
|
||||
* deletes or docs were flushed) if necessary
|
||||
* @param flushDocStores if false we are allowed to keep
|
||||
* doc stores open to share with the next segment
|
||||
* @param flushDeletes whether pending deletes should also
|
||||
* be flushed
|
||||
*/
|
||||
protected final void flush(boolean triggerMerge, boolean flushDocStores, boolean flushDeletes) throws CorruptIndexException, IOException {
|
||||
protected final void flush(boolean triggerMerge, boolean flushDeletes) throws CorruptIndexException, IOException {
|
||||
|
||||
// NOTE: this method cannot be sync'd because
|
||||
// maybeMerge() in turn calls mergeScheduler.merge which
|
||||
|
@ -2481,7 +2447,7 @@ public class IndexWriter implements Closeable {
|
|||
|
||||
// We can be called during close, when closing==true, so we must pass false to ensureOpen:
|
||||
ensureOpen(false);
|
||||
if (doFlush(flushDocStores, flushDeletes) && triggerMerge) {
|
||||
if (doFlush(flushDeletes) && triggerMerge) {
|
||||
maybeMerge();
|
||||
}
|
||||
}
|
||||
|
@ -2489,7 +2455,7 @@ public class IndexWriter implements Closeable {
|
|||
// TODO: this method should not have to be entirely
|
||||
// synchronized, ie, merges should be allowed to commit
|
||||
// even while a flush is happening
|
||||
private synchronized final boolean doFlush(boolean closeDocStores, boolean applyAllDeletes) throws CorruptIndexException, IOException {
|
||||
private synchronized boolean doFlush(boolean applyAllDeletes) throws CorruptIndexException, IOException {
|
||||
|
||||
if (hitOOM) {
|
||||
throw new IllegalStateException("this writer hit an OutOfMemoryError; cannot flush");
|
||||
|
@ -2512,11 +2478,11 @@ public class IndexWriter implements Closeable {
|
|||
try {
|
||||
|
||||
if (infoStream != null) {
|
||||
message(" start flush: applyAllDeletes=" + applyAllDeletes + " closeDocStores=" + closeDocStores);
|
||||
message(" start flush: applyAllDeletes=" + applyAllDeletes);
|
||||
message(" index before flush " + segString());
|
||||
}
|
||||
|
||||
final SegmentInfo newSegment = docWriter.flush(this, closeDocStores, deleter, mergePolicy, segmentInfos);
|
||||
final SegmentInfo newSegment = docWriter.flush(this, deleter, mergePolicy, segmentInfos);
|
||||
if (newSegment != null) {
|
||||
setDiagnostics(newSegment, "flush");
|
||||
segmentInfos.add(newSegment);
|
||||
|
@ -2564,18 +2530,9 @@ public class IndexWriter implements Closeable {
|
|||
return false;
|
||||
} finally {
|
||||
flushControl.clearFlushPending();
|
||||
if (!success) {
|
||||
if (infoStream != null) {
|
||||
if (!success && infoStream != null)
|
||||
message("hit exception during flush");
|
||||
}
|
||||
if (docWriter != null) {
|
||||
final Collection<String> files = docWriter.abortedFiles();
|
||||
if (files != null) {
|
||||
deleter.deleteNewFiles(files);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/** Expert: Return the total size of all index files currently cached in memory.
|
||||
|
@ -2590,7 +2547,7 @@ public class IndexWriter implements Closeable {
|
|||
* buffered in RAM. */
|
||||
public final synchronized int numRamDocs() {
|
||||
ensureOpen();
|
||||
return docWriter.getNumDocsInRAM();
|
||||
return docWriter.getNumDocs();
|
||||
}
|
||||
|
||||
private int ensureContiguousMerge(MergePolicy.OneMerge merge) {
|
||||
|
@ -2695,7 +2652,7 @@ public class IndexWriter implements Closeable {
|
|||
}
|
||||
|
||||
/* FIXME if we want to support non-contiguous segment merges */
|
||||
synchronized private boolean commitMerge(MergePolicy.OneMerge merge, SegmentMerger merger, SegmentReader mergedReader) throws IOException {
|
||||
synchronized private boolean commitMerge(MergePolicy.OneMerge merge, SegmentReader mergedReader) throws IOException {
|
||||
|
||||
assert testPoint("startCommitMerge");
|
||||
|
||||
|
@ -2925,7 +2882,7 @@ public class IndexWriter implements Closeable {
|
|||
}
|
||||
}
|
||||
|
||||
final synchronized private void _mergeInit(MergePolicy.OneMerge merge) throws IOException {
|
||||
synchronized private void _mergeInit(MergePolicy.OneMerge merge) throws IOException {
|
||||
|
||||
assert testPoint("startMergeInit");
|
||||
|
||||
|
@ -2943,132 +2900,25 @@ public class IndexWriter implements Closeable {
|
|||
if (merge.isAborted())
|
||||
return;
|
||||
|
||||
final SegmentInfos sourceSegments = merge.segments;
|
||||
final int end = sourceSegments.size();
|
||||
|
||||
// Check whether this merge will allow us to skip
|
||||
// merging the doc stores (stored field & vectors).
|
||||
// This is a very substantial optimization (saves tons
|
||||
// of IO).
|
||||
|
||||
Directory lastDir = directory;
|
||||
String lastDocStoreSegment = null;
|
||||
int next = -1;
|
||||
|
||||
boolean mergeDocStores = false;
|
||||
boolean doFlushDocStore = false;
|
||||
boolean hasVectors = false;
|
||||
final String currentDocStoreSegment = docWriter.getDocStoreSegment();
|
||||
|
||||
// Test each segment to be merged: check if we need to
|
||||
// flush/merge doc stores
|
||||
for (int i = 0; i < end; i++) {
|
||||
SegmentInfo si = sourceSegments.info(i);
|
||||
|
||||
// If it has deletions we must merge the doc stores
|
||||
if (si.hasDeletions())
|
||||
mergeDocStores = true;
|
||||
|
||||
if (si.getHasVectors()) {
|
||||
for (SegmentInfo sourceSegment : merge.segments) {
|
||||
if (sourceSegment.getHasVectors()) {
|
||||
hasVectors = true;
|
||||
}
|
||||
|
||||
// If it has its own (private) doc stores we must
|
||||
// merge the doc stores
|
||||
if (-1 == si.getDocStoreOffset())
|
||||
mergeDocStores = true;
|
||||
|
||||
// If it has a different doc store segment than
|
||||
// previous segments, we must merge the doc stores
|
||||
String docStoreSegment = si.getDocStoreSegment();
|
||||
if (docStoreSegment == null)
|
||||
mergeDocStores = true;
|
||||
else if (lastDocStoreSegment == null)
|
||||
lastDocStoreSegment = docStoreSegment;
|
||||
else if (!lastDocStoreSegment.equals(docStoreSegment))
|
||||
mergeDocStores = true;
|
||||
|
||||
// Segments' docScoreOffsets must be in-order,
|
||||
// contiguous. For the default merge policy now
|
||||
// this will always be the case but for an arbitrary
|
||||
// merge policy this may not be the case
|
||||
if (-1 == next)
|
||||
next = si.getDocStoreOffset() + si.docCount;
|
||||
else if (next != si.getDocStoreOffset())
|
||||
mergeDocStores = true;
|
||||
else
|
||||
next = si.getDocStoreOffset() + si.docCount;
|
||||
|
||||
// If the segment comes from a different directory
|
||||
// we must merge
|
||||
if (lastDir != si.dir)
|
||||
mergeDocStores = true;
|
||||
|
||||
// If the segment is referencing the current "live"
|
||||
// doc store outputs then we must merge
|
||||
if (si.getDocStoreOffset() != -1 && currentDocStoreSegment != null && si.getDocStoreSegment().equals(currentDocStoreSegment)) {
|
||||
doFlushDocStore = true;
|
||||
}
|
||||
}
|
||||
|
||||
// if a mergedSegmentWarmer is installed, we must merge
|
||||
// the doc stores because we will open a full
|
||||
// SegmentReader on the merged segment:
|
||||
if (!mergeDocStores && mergedSegmentWarmer != null && currentDocStoreSegment != null && lastDocStoreSegment != null && lastDocStoreSegment.equals(currentDocStoreSegment)) {
|
||||
mergeDocStores = true;
|
||||
}
|
||||
|
||||
final int docStoreOffset;
|
||||
final String docStoreSegment;
|
||||
final boolean docStoreIsCompoundFile;
|
||||
|
||||
if (mergeDocStores) {
|
||||
docStoreOffset = -1;
|
||||
docStoreSegment = null;
|
||||
docStoreIsCompoundFile = false;
|
||||
} else {
|
||||
SegmentInfo si = sourceSegments.info(0);
|
||||
docStoreOffset = si.getDocStoreOffset();
|
||||
docStoreSegment = si.getDocStoreSegment();
|
||||
docStoreIsCompoundFile = si.getDocStoreIsCompoundFile();
|
||||
}
|
||||
|
||||
if (mergeDocStores && doFlushDocStore) {
|
||||
// SegmentMerger intends to merge the doc stores
|
||||
// (stored fields, vectors), and at least one of the
|
||||
// segments to be merged refers to the currently
|
||||
// live doc stores.
|
||||
|
||||
// TODO: if we know we are about to merge away these
|
||||
// newly flushed doc store files then we should not
|
||||
// make compound file out of them...
|
||||
if (infoStream != null)
|
||||
message("now flush at merge");
|
||||
doFlush(true, false);
|
||||
updatePendingMerges(1, false);
|
||||
}
|
||||
|
||||
merge.mergeDocStores = mergeDocStores;
|
||||
|
||||
// Bind a new segment name here so even with
|
||||
// ConcurrentMergePolicy we keep deterministic segment
|
||||
// names.
|
||||
merge.info = new SegmentInfo(newSegmentName(), 0,
|
||||
directory, false, docStoreOffset,
|
||||
docStoreSegment,
|
||||
docStoreIsCompoundFile,
|
||||
false,
|
||||
null,
|
||||
hasVectors);
|
||||
merge.info = new SegmentInfo(newSegmentName(), 0, directory, false, false, null, hasVectors);
|
||||
|
||||
Map<String,String> details = new HashMap<String,String>();
|
||||
details.put("optimize", Boolean.toString(merge.optimize));
|
||||
details.put("mergeFactor", Integer.toString(end));
|
||||
details.put("mergeDocStores", Boolean.toString(mergeDocStores));
|
||||
details.put("mergeFactor", Integer.toString(merge.segments.size()));
|
||||
setDiagnostics(merge.info, "merge", details);
|
||||
|
||||
if (infoStream != null) {
|
||||
message("merge seg=" + merge.info.name + " mergeDocStores=" + mergeDocStores);
|
||||
message("merge seg=" + merge.info.name);
|
||||
}
|
||||
|
||||
// Also enroll the merged segment into mergingSegments;
|
||||
|
@ -3192,8 +3042,6 @@ public class IndexWriter implements Closeable {
|
|||
|
||||
final String mergedName = merge.info.name;
|
||||
|
||||
SegmentMerger merger = null;
|
||||
|
||||
int mergedDocCount = 0;
|
||||
|
||||
SegmentInfos sourceSegments = merge.segments;
|
||||
|
@ -3202,20 +3050,11 @@ public class IndexWriter implements Closeable {
|
|||
if (infoStream != null)
|
||||
message("merging " + merge.segString(directory));
|
||||
|
||||
merger = new SegmentMerger(directory, termIndexInterval, mergedName, merge, codecs, payloadProcessorProvider);
|
||||
SegmentMerger merger = new SegmentMerger(directory, termIndexInterval, mergedName, merge, codecs, payloadProcessorProvider);
|
||||
|
||||
merge.readers = new SegmentReader[numSegments];
|
||||
merge.readersClone = new SegmentReader[numSegments];
|
||||
|
||||
boolean mergeDocStores = false;
|
||||
|
||||
final String currentDocStoreSegment;
|
||||
synchronized(this) {
|
||||
currentDocStoreSegment = docWriter.getDocStoreSegment();
|
||||
}
|
||||
|
||||
boolean currentDSSMerged = false;
|
||||
|
||||
// This is try/finally to make sure merger's readers are
|
||||
// closed:
|
||||
boolean success = false;
|
||||
|
@ -3227,7 +3066,7 @@ public class IndexWriter implements Closeable {
|
|||
|
||||
// Hold onto the "live" reader; we will use this to
|
||||
// commit merged deletes
|
||||
SegmentReader reader = merge.readers[i] = readerPool.get(info, merge.mergeDocStores,
|
||||
SegmentReader reader = merge.readers[i] = readerPool.get(info, true,
|
||||
MERGE_READ_BUFFER_SIZE,
|
||||
-config.getReaderTermsIndexDivisor());
|
||||
|
||||
|
@ -3237,14 +3076,6 @@ public class IndexWriter implements Closeable {
|
|||
SegmentReader clone = merge.readersClone[i] = (SegmentReader) reader.clone(true);
|
||||
merger.add(clone);
|
||||
|
||||
if (clone.hasDeletions()) {
|
||||
mergeDocStores = true;
|
||||
}
|
||||
|
||||
if (info.getDocStoreOffset() != -1 && currentDocStoreSegment != null) {
|
||||
currentDSSMerged |= currentDocStoreSegment.equals(info.getDocStoreSegment());
|
||||
}
|
||||
|
||||
totDocCount += clone.numDocs();
|
||||
}
|
||||
|
||||
|
@ -3254,39 +3085,8 @@ public class IndexWriter implements Closeable {
|
|||
|
||||
merge.checkAborted(directory);
|
||||
|
||||
// If deletions have arrived and it has now become
|
||||
// necessary to merge doc stores, go and open them:
|
||||
if (mergeDocStores && !merge.mergeDocStores) {
|
||||
merge.mergeDocStores = true;
|
||||
synchronized(this) {
|
||||
|
||||
// If 1) we must now merge doc stores, and 2) at
|
||||
// least one of the segments we are merging uses
|
||||
// the doc store we are now writing to, we must at
|
||||
// this point force this doc store closed (by
|
||||
// calling flush). If we didn't do this then the
|
||||
// readers will attempt to open an IndexInput
|
||||
// on files that have still-open IndexOutputs
|
||||
// against them:
|
||||
if (currentDSSMerged) {
|
||||
if (infoStream != null) {
|
||||
message("now flush at mergeMiddle");
|
||||
}
|
||||
doFlush(true, false);
|
||||
updatePendingMerges(1, false);
|
||||
}
|
||||
}
|
||||
|
||||
for(int i=0;i<numSegments;i++) {
|
||||
merge.readersClone[i].openDocStores();
|
||||
}
|
||||
|
||||
// Clear DSS
|
||||
merge.info.setDocStore(-1, null, false);
|
||||
}
|
||||
|
||||
// This is where all the work happens:
|
||||
mergedDocCount = merge.info.docCount = merger.merge(merge.mergeDocStores);
|
||||
mergedDocCount = merge.info.docCount = merger.merge();
|
||||
|
||||
// Record which codec was used to write the segment
|
||||
merge.info.setSegmentCodecs(merger.getSegmentCodecs());
|
||||
|
@ -3366,12 +3166,7 @@ public class IndexWriter implements Closeable {
|
|||
final int termsIndexDivisor;
|
||||
final boolean loadDocStores;
|
||||
|
||||
// if the merged segment warmer was not installed when
|
||||
// this merge was started, causing us to not force
|
||||
// the docStores to close, we can't warm it now
|
||||
final boolean canWarm = merge.info.getDocStoreSegment() == null || currentDocStoreSegment == null || !merge.info.getDocStoreSegment().equals(currentDocStoreSegment);
|
||||
|
||||
if (poolReaders && mergedSegmentWarmer != null && canWarm) {
|
||||
if (poolReaders && mergedSegmentWarmer != null) {
|
||||
// Load terms index & doc stores so the segment
|
||||
// warmer can run searches, load documents/term
|
||||
// vectors
|
||||
|
@ -3392,7 +3187,7 @@ public class IndexWriter implements Closeable {
|
|||
mergedSegmentWarmer.warm(mergedReader);
|
||||
}
|
||||
|
||||
if (!commitMerge(merge, merger, mergedReader)) {
|
||||
if (!commitMerge(merge, mergedReader)) {
|
||||
// commitMerge will return false if this merge was aborted
|
||||
return 0;
|
||||
}
|
||||
|
@ -3523,7 +3318,7 @@ public class IndexWriter implements Closeable {
|
|||
}
|
||||
|
||||
// First, we clone & incref the segmentInfos we intend
|
||||
// to sync, then, without locking, we sync() each file
|
||||
// to sync, then, without locking, we sync() all files
|
||||
// referenced by toSync, in the background.
|
||||
|
||||
if (infoStream != null)
|
||||
|
@ -3531,25 +3326,7 @@ public class IndexWriter implements Closeable {
|
|||
|
||||
readerPool.commit();
|
||||
|
||||
// It's possible another flush (that did not close
|
||||
// the open do stores) snuck in after the flush we
|
||||
// just did, so we remove any tail segments
|
||||
// referencing the open doc store from the
|
||||
// SegmentInfos we are about to sync (the main
|
||||
// SegmentInfos will keep them):
|
||||
toSync = (SegmentInfos) segmentInfos.clone();
|
||||
final String dss = docWriter.getDocStoreSegment();
|
||||
if (dss != null) {
|
||||
while(true) {
|
||||
final String dss2 = toSync.info(toSync.size()-1).getDocStoreSegment();
|
||||
if (dss2 == null || !dss2.equals(dss)) {
|
||||
break;
|
||||
}
|
||||
toSync.remove(toSync.size()-1);
|
||||
changeCount++;
|
||||
segmentInfos.changed();
|
||||
}
|
||||
}
|
||||
assert filesExist(toSync);
|
||||
|
||||
if (commitUserData != null)
|
||||
|
|
|
@ -32,9 +32,6 @@ abstract class InvertedDocConsumer {
|
|||
/** Flush a new segment */
|
||||
abstract void flush(Map<InvertedDocConsumerPerThread,Collection<InvertedDocConsumerPerField>> threadsAndFields, SegmentWriteState state) throws IOException;
|
||||
|
||||
/** Close doc stores */
|
||||
abstract void closeDocStore(SegmentWriteState state) throws IOException;
|
||||
|
||||
/** Attempt to free RAM, returning true if any RAM was
|
||||
* freed */
|
||||
abstract boolean freeRAM();
|
||||
|
|
|
@ -24,7 +24,6 @@ import java.io.IOException;
|
|||
abstract class InvertedDocEndConsumer {
|
||||
abstract InvertedDocEndConsumerPerThread addThread(DocInverterPerThread docInverterPerThread);
|
||||
abstract void flush(Map<InvertedDocEndConsumerPerThread,Collection<InvertedDocEndConsumerPerField>> threadsAndFields, SegmentWriteState state) throws IOException;
|
||||
abstract void closeDocStore(SegmentWriteState state) throws IOException;
|
||||
abstract void abort();
|
||||
abstract void setFieldInfos(FieldInfos fieldInfos);
|
||||
}
|
||||
|
|
|
@ -70,7 +70,6 @@ public abstract class LogMergePolicy extends MergePolicy {
|
|||
protected boolean calibrateSizeByDeletes = true;
|
||||
|
||||
protected boolean useCompoundFile = true;
|
||||
protected boolean useCompoundDocStore = true;
|
||||
|
||||
public LogMergePolicy() {
|
||||
super();
|
||||
|
@ -157,27 +156,6 @@ public abstract class LogMergePolicy extends MergePolicy {
|
|||
return useCompoundFile;
|
||||
}
|
||||
|
||||
// Javadoc inherited
|
||||
@Override
|
||||
public boolean useCompoundDocStore(SegmentInfos infos) {
|
||||
return useCompoundDocStore;
|
||||
}
|
||||
|
||||
/** Sets whether compound file format should be used for
|
||||
* newly flushed and newly merged doc store
|
||||
* segment files (term vectors and stored fields). */
|
||||
public void setUseCompoundDocStore(boolean useCompoundDocStore) {
|
||||
this.useCompoundDocStore = useCompoundDocStore;
|
||||
}
|
||||
|
||||
/** Returns true if newly flushed and newly merge doc
|
||||
* store segment files (term vectors and stored fields)
|
||||
* are written in compound file format. @see
|
||||
* #setUseCompoundDocStore */
|
||||
public boolean getUseCompoundDocStore() {
|
||||
return useCompoundDocStore;
|
||||
}
|
||||
|
||||
/** Sets whether the segment size should be calibrated by
|
||||
* the number of deletes when choosing segments for merge. */
|
||||
public void setCalibrateSizeByDeletes(boolean calibrateSizeByDeletes) {
|
||||
|
@ -591,8 +569,7 @@ public abstract class LogMergePolicy extends MergePolicy {
|
|||
sb.append("maxMergeSize=").append(maxMergeSize).append(", ");
|
||||
sb.append("calibrateSizeByDeletes=").append(calibrateSizeByDeletes).append(", ");
|
||||
sb.append("maxMergeDocs=").append(maxMergeDocs).append(", ");
|
||||
sb.append("useCompoundFile=").append(useCompoundFile).append(", ");
|
||||
sb.append("useCompoundDocStore=").append(useCompoundDocStore);
|
||||
sb.append("useCompoundFile=").append(useCompoundFile);
|
||||
sb.append("]");
|
||||
return sb.toString();
|
||||
}
|
||||
|
|
|
@ -67,7 +67,6 @@ public abstract class MergePolicy implements java.io.Closeable {
|
|||
public static class OneMerge {
|
||||
|
||||
SegmentInfo info; // used by IndexWriter
|
||||
boolean mergeDocStores; // used by IndexWriter
|
||||
boolean optimize; // used by IndexWriter
|
||||
boolean registerDone; // used by IndexWriter
|
||||
long mergeGen; // used by IndexWriter
|
||||
|
@ -153,9 +152,6 @@ public abstract class MergePolicy implements java.io.Closeable {
|
|||
b.append(" into ").append(info.name);
|
||||
if (optimize)
|
||||
b.append(" [optimize]");
|
||||
if (mergeDocStores) {
|
||||
b.append(" [mergeDocStores]");
|
||||
}
|
||||
if (aborted) {
|
||||
b.append(" [ABORTED]");
|
||||
}
|
||||
|
@ -318,10 +314,4 @@ public abstract class MergePolicy implements java.io.Closeable {
|
|||
* Returns true if a new segment (regardless of its origin) should use the compound file format.
|
||||
*/
|
||||
public abstract boolean useCompoundFile(SegmentInfos segments, SegmentInfo newSegment) throws IOException;
|
||||
|
||||
/**
|
||||
* Returns true if the doc store files should use the
|
||||
* compound file format.
|
||||
*/
|
||||
public abstract boolean useCompoundDocStore(SegmentInfos segments);
|
||||
}
|
||||
|
|
|
@ -66,9 +66,6 @@ public final class NoMergePolicy extends MergePolicy {
|
|||
public MergeSpecification findMergesToExpungeDeletes(SegmentInfos segmentInfos)
|
||||
throws CorruptIndexException, IOException { return null; }
|
||||
|
||||
@Override
|
||||
public boolean useCompoundDocStore(SegmentInfos segments) { return useCompoundFile; }
|
||||
|
||||
@Override
|
||||
public boolean useCompoundFile(SegmentInfos segments, SegmentInfo newSegment) { return useCompoundFile; }
|
||||
|
||||
|
|
|
@ -171,7 +171,4 @@ final class NormsWriter extends InvertedDocEndConsumer {
|
|||
normsOut.close();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
void closeDocStore(SegmentWriteState state) {}
|
||||
}
|
||||
|
|
|
@ -86,22 +86,18 @@ public final class SegmentInfo {
|
|||
|
||||
private Map<String,String> diagnostics;
|
||||
|
||||
public SegmentInfo(String name, int docCount, Directory dir, boolean isCompoundFile, int docStoreOffset,
|
||||
String docStoreSegment, boolean docStoreIsCompoundFile, boolean hasProx, SegmentCodecs segmentCodecs,
|
||||
boolean hasVectors) {
|
||||
public SegmentInfo(String name, int docCount, Directory dir, boolean isCompoundFile,
|
||||
boolean hasProx, SegmentCodecs segmentCodecs, boolean hasVectors) {
|
||||
this.name = name;
|
||||
this.docCount = docCount;
|
||||
this.dir = dir;
|
||||
delGen = NO;
|
||||
this.isCompoundFile = isCompoundFile;
|
||||
this.docStoreOffset = docStoreOffset;
|
||||
this.docStoreSegment = docStoreSegment;
|
||||
this.docStoreIsCompoundFile = docStoreIsCompoundFile;
|
||||
this.docStoreOffset = -1;
|
||||
this.hasProx = hasProx;
|
||||
this.segmentCodecs = segmentCodecs;
|
||||
this.hasVectors = hasVectors;
|
||||
delCount = 0;
|
||||
assert docStoreOffset == -1 || docStoreSegment != null: "dso=" + docStoreOffset + " dss=" + docStoreSegment + " docCount=" + docCount;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -274,7 +270,10 @@ public final class SegmentInfo {
|
|||
|
||||
@Override
|
||||
public Object clone() {
|
||||
SegmentInfo si = new SegmentInfo(name, docCount, dir, isCompoundFile, docStoreOffset, docStoreSegment, docStoreIsCompoundFile, hasProx, segmentCodecs, false);
|
||||
SegmentInfo si = new SegmentInfo(name, docCount, dir, isCompoundFile, hasProx, segmentCodecs, false);
|
||||
si.docStoreOffset = docStoreOffset;
|
||||
si.docStoreSegment = docStoreSegment;
|
||||
si.docStoreIsCompoundFile = docStoreIsCompoundFile;
|
||||
si.delGen = delGen;
|
||||
si.delCount = delCount;
|
||||
si.diagnostics = new HashMap<String, String>(diagnostics);
|
||||
|
|
|
@ -62,12 +62,6 @@ final class SegmentMerger {
|
|||
|
||||
private final CheckAbort checkAbort;
|
||||
|
||||
// Whether we should merge doc stores (stored fields and
|
||||
// vectors files). When all segments we are merging
|
||||
// already share the same doc store files, we don't need
|
||||
// to merge the doc stores.
|
||||
private boolean mergeDocStores;
|
||||
|
||||
/** Maximum number of contiguous documents to bulk-copy
|
||||
when merging stored fields */
|
||||
private final static int MAX_RAW_MERGE_DOCS = 4192;
|
||||
|
@ -115,22 +109,6 @@ final class SegmentMerger {
|
|||
* @throws IOException if there is a low-level IO error
|
||||
*/
|
||||
final int merge() throws CorruptIndexException, IOException {
|
||||
return merge(true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Merges the readers specified by the {@link #add} method
|
||||
* into the directory passed to the constructor.
|
||||
* @param mergeDocStores if false, we will not merge the
|
||||
* stored fields nor vectors files
|
||||
* @return The number of documents that were merged
|
||||
* @throws CorruptIndexException if the index is corrupt
|
||||
* @throws IOException if there is a low-level IO error
|
||||
*/
|
||||
final int merge(boolean mergeDocStores) throws CorruptIndexException, IOException {
|
||||
|
||||
this.mergeDocStores = mergeDocStores;
|
||||
|
||||
// NOTE: it's important to add calls to
|
||||
// checkAbort.work(...) if you make any changes to this
|
||||
// method that will spend alot of time. The frequency
|
||||
|
@ -142,9 +120,8 @@ final class SegmentMerger {
|
|||
mergeTerms();
|
||||
mergeNorms();
|
||||
|
||||
if (mergeDocStores && fieldInfos.hasVectors()) {
|
||||
if (fieldInfos.hasVectors())
|
||||
mergeVectors();
|
||||
}
|
||||
|
||||
return mergedDocs;
|
||||
}
|
||||
|
@ -154,8 +131,6 @@ final class SegmentMerger {
|
|||
|
||||
// Basic files
|
||||
for (String ext : IndexFileNames.COMPOUND_EXTENSIONS_NOT_CODEC) {
|
||||
if (mergeDocStores || (!ext.equals(IndexFileNames.FIELDS_EXTENSION) &&
|
||||
!ext.equals(IndexFileNames.FIELDS_INDEX_EXTENSION)))
|
||||
fileSet.add(IndexFileNames.segmentFileName(segment, "", ext));
|
||||
}
|
||||
|
||||
|
@ -172,7 +147,7 @@ final class SegmentMerger {
|
|||
}
|
||||
|
||||
// Vector files
|
||||
if (fieldInfos.hasVectors() && mergeDocStores) {
|
||||
if (fieldInfos.hasVectors()) {
|
||||
for (String ext : IndexFileNames.VECTOR_EXTENSIONS) {
|
||||
fileSet.add(IndexFileNames.segmentFileName(segment, "", ext));
|
||||
}
|
||||
|
@ -251,18 +226,8 @@ final class SegmentMerger {
|
|||
* @throws CorruptIndexException if the index is corrupt
|
||||
* @throws IOException if there is a low-level IO error
|
||||
*/
|
||||
private final int mergeFields() throws CorruptIndexException, IOException {
|
||||
|
||||
if (!mergeDocStores) {
|
||||
// When we are not merging by doc stores, their field
|
||||
// name -> number mapping are the same. So, we start
|
||||
// with the fieldInfos of the last segment in this
|
||||
// case, to keep that numbering.
|
||||
final SegmentReader sr = (SegmentReader) readers.get(readers.size()-1);
|
||||
fieldInfos = (FieldInfos) sr.core.fieldInfos.clone();
|
||||
} else {
|
||||
private int mergeFields() throws CorruptIndexException, IOException {
|
||||
fieldInfos = new FieldInfos();// merge field names
|
||||
}
|
||||
|
||||
for (IndexReader reader : readers) {
|
||||
if (reader instanceof SegmentReader) {
|
||||
|
@ -294,8 +259,6 @@ final class SegmentMerger {
|
|||
|
||||
setMatchingSegmentReaders();
|
||||
|
||||
if (mergeDocStores) {
|
||||
// merge field values
|
||||
final FieldsWriter fieldsWriter = new FieldsWriter(directory, segment, fieldInfos);
|
||||
|
||||
try {
|
||||
|
@ -332,16 +295,7 @@ final class SegmentMerger {
|
|||
// 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");
|
||||
|
||||
} else {
|
||||
// If we are skipping the doc stores, that means there
|
||||
// are no deletions in any of these segments, so we
|
||||
// just sum numDocs() of each segment to get total docCount
|
||||
for (final IndexReader reader : readers) {
|
||||
docCount += reader.numDocs();
|
||||
}
|
||||
}
|
||||
|
||||
segmentWriteState = new SegmentWriteState(null, directory, segment, fieldInfos, null, docCount, 0, termIndexInterval, codecInfo);
|
||||
segmentWriteState = new SegmentWriteState(null, directory, segment, fieldInfos, docCount, termIndexInterval, codecInfo);
|
||||
|
||||
return docCount;
|
||||
}
|
||||
|
|
|
@ -31,9 +31,7 @@ public class SegmentWriteState {
|
|||
public final Directory directory;
|
||||
public final String segmentName;
|
||||
public final FieldInfos fieldInfos;
|
||||
public final String docStoreSegmentName;
|
||||
public final int numDocs;
|
||||
public int numDocsInStore;
|
||||
public boolean hasVectors;
|
||||
public final Collection<String> flushedFiles;
|
||||
|
||||
|
@ -62,15 +60,12 @@ public class SegmentWriteState {
|
|||
|
||||
|
||||
public SegmentWriteState(PrintStream infoStream, Directory directory, String segmentName, FieldInfos fieldInfos,
|
||||
String docStoreSegmentName, int numDocs,
|
||||
int numDocsInStore, int termIndexInterval, SegmentCodecs segmentCodecs) {
|
||||
int numDocs, int termIndexInterval, SegmentCodecs segmentCodecs) {
|
||||
this.infoStream = infoStream;
|
||||
this.directory = directory;
|
||||
this.segmentName = segmentName;
|
||||
this.fieldInfos = fieldInfos;
|
||||
this.docStoreSegmentName = docStoreSegmentName;
|
||||
this.numDocs = numDocs;
|
||||
this.numDocsInStore = numDocsInStore;
|
||||
this.termIndexInterval = termIndexInterval;
|
||||
this.segmentCodecs = segmentCodecs;
|
||||
flushedFiles = new HashSet<String>();
|
||||
|
@ -85,9 +80,7 @@ public class SegmentWriteState {
|
|||
directory = state.directory;
|
||||
segmentName = state.segmentName;
|
||||
fieldInfos = state.fieldInfos;
|
||||
docStoreSegmentName = state.docStoreSegmentName;
|
||||
numDocs = state.numDocs;
|
||||
numDocsInStore = state.numDocsInStore;
|
||||
termIndexInterval = state.termIndexInterval;
|
||||
segmentCodecs = state.segmentCodecs;
|
||||
flushedFiles = state.flushedFiles;
|
||||
|
|
|
@ -29,7 +29,6 @@ final class StoredFieldsWriter {
|
|||
final DocumentsWriter docWriter;
|
||||
final FieldInfos fieldInfos;
|
||||
int lastDocID;
|
||||
private String docStoreSegment;
|
||||
|
||||
PerDoc[] docFreeList = new PerDoc[1];
|
||||
int freeCount;
|
||||
|
@ -44,60 +43,31 @@ final class StoredFieldsWriter {
|
|||
}
|
||||
|
||||
synchronized public void flush(SegmentWriteState state) throws IOException {
|
||||
|
||||
if (state.numDocsInStore > 0) {
|
||||
// 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:
|
||||
if (state.numDocs > lastDocID) {
|
||||
initFieldsWriter();
|
||||
|
||||
// Fill fdx file to include any final docs that we
|
||||
// skipped because they hit non-aborting exceptions
|
||||
fill(state.numDocsInStore - docWriter.getDocStoreOffset());
|
||||
}
|
||||
|
||||
if (fieldsWriter != null)
|
||||
fieldsWriter.flush();
|
||||
}
|
||||
|
||||
private synchronized void initFieldsWriter() throws IOException {
|
||||
if (fieldsWriter == null) {
|
||||
docStoreSegment = docWriter.getDocStoreSegment();
|
||||
if (docStoreSegment != null) {
|
||||
fieldsWriter = new FieldsWriter(docWriter.directory,
|
||||
docStoreSegment,
|
||||
fieldInfos);
|
||||
docWriter.addOpenFile(IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.FIELDS_EXTENSION));
|
||||
docWriter.addOpenFile(IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.FIELDS_INDEX_EXTENSION));
|
||||
lastDocID = 0;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
synchronized public void closeDocStore(SegmentWriteState state) throws IOException {
|
||||
final int inc = state.numDocsInStore - lastDocID;
|
||||
if (inc > 0) {
|
||||
initFieldsWriter();
|
||||
fill(state.numDocsInStore - docWriter.getDocStoreOffset());
|
||||
fill(state.numDocs);
|
||||
}
|
||||
|
||||
if (fieldsWriter != null) {
|
||||
fieldsWriter.close();
|
||||
fieldsWriter = null;
|
||||
assert docStoreSegment != null;
|
||||
assert state.docStoreSegmentName != null;
|
||||
assert docStoreSegment.equals(state.docStoreSegmentName): "fieldsWriter wrote to segment=" + docStoreSegment + " vs SegmentWriteState segment=" + state.docStoreSegmentName;
|
||||
lastDocID = 0;
|
||||
String fieldsName = IndexFileNames.segmentFileName(state.docStoreSegmentName, "", IndexFileNames.FIELDS_EXTENSION);
|
||||
String fieldsIdxName = IndexFileNames.segmentFileName(state.docStoreSegmentName, "", IndexFileNames.FIELDS_INDEX_EXTENSION);
|
||||
|
||||
String fieldsName = IndexFileNames.segmentFileName(state.segmentName, "", IndexFileNames.FIELDS_EXTENSION);
|
||||
String fieldsIdxName = IndexFileNames.segmentFileName(state.segmentName, "", IndexFileNames.FIELDS_INDEX_EXTENSION);
|
||||
state.flushedFiles.add(fieldsName);
|
||||
state.flushedFiles.add(fieldsIdxName);
|
||||
|
||||
docWriter.removeOpenFile(fieldsName);
|
||||
docWriter.removeOpenFile(fieldsIdxName);
|
||||
if (4 + ((long) state.numDocs) * 8 != state.directory.fileLength(fieldsIdxName)) {
|
||||
throw new RuntimeException("after flush: fdx size mismatch: " + state.numDocs + " docs vs " + state.directory.fileLength(fieldsIdxName) + " length in bytes of " + fieldsIdxName + " file exists?=" + state.directory.fileExists(fieldsIdxName));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (4+((long) state.numDocsInStore)*8 != state.directory.fileLength(fieldsIdxName))
|
||||
throw new RuntimeException("after flush: fdx size mismatch: " + state.numDocsInStore + " docs vs " + state.directory.fileLength(fieldsIdxName) + " length in bytes of " + fieldsIdxName + " file exists?=" + state.directory.fileExists(fieldsIdxName));
|
||||
private synchronized void initFieldsWriter() throws IOException {
|
||||
if (fieldsWriter == null) {
|
||||
fieldsWriter = new FieldsWriter(docWriter.directory, docWriter.getSegment(), fieldInfos);
|
||||
lastDocID = 0;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -114,16 +84,14 @@ final class StoredFieldsWriter {
|
|||
docFreeList = new PerDoc[ArrayUtil.oversize(allocCount, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
|
||||
}
|
||||
return new PerDoc();
|
||||
} else
|
||||
} else {
|
||||
return docFreeList[--freeCount];
|
||||
}
|
||||
}
|
||||
|
||||
synchronized void abort() {
|
||||
if (fieldsWriter != null) {
|
||||
try {
|
||||
fieldsWriter.close();
|
||||
} catch (Throwable t) {
|
||||
}
|
||||
fieldsWriter.abort();
|
||||
fieldsWriter = null;
|
||||
lastDocID = 0;
|
||||
}
|
||||
|
@ -131,12 +99,9 @@ final class StoredFieldsWriter {
|
|||
|
||||
/** Fills in any hole in the docIDs */
|
||||
void fill(int docID) throws IOException {
|
||||
final int docStoreOffset = docWriter.getDocStoreOffset();
|
||||
|
||||
// We must "catch up" for all docs before us
|
||||
// that had no stored fields:
|
||||
final int end = docID+docStoreOffset;
|
||||
while(lastDocID < end) {
|
||||
while(lastDocID < docID) {
|
||||
fieldsWriter.skipDocument();
|
||||
lastDocID++;
|
||||
}
|
||||
|
@ -156,10 +121,6 @@ final class StoredFieldsWriter {
|
|||
assert docWriter.writer.testPoint("StoredFieldsWriter.finishDocument end");
|
||||
}
|
||||
|
||||
public boolean freeRAM() {
|
||||
return false;
|
||||
}
|
||||
|
||||
synchronized void free(PerDoc perDoc) {
|
||||
assert freeCount < docFreeList.length;
|
||||
assert 0 == perDoc.numStoredFields;
|
||||
|
|
|
@ -20,6 +20,7 @@ package org.apache.lucene.index;
|
|||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.store.RAMOutputStream;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -30,7 +31,6 @@ import java.util.Map;
|
|||
final class TermVectorsTermsWriter extends TermsHashConsumer {
|
||||
|
||||
final DocumentsWriter docWriter;
|
||||
TermVectorsWriter termVectorsWriter;
|
||||
PerDoc[] docFreeList = new PerDoc[1];
|
||||
int freeCount;
|
||||
IndexOutput tvx;
|
||||
|
@ -50,26 +50,29 @@ final class TermVectorsTermsWriter extends TermsHashConsumer {
|
|||
|
||||
@Override
|
||||
synchronized void flush(Map<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> threadsAndFields, final SegmentWriteState state) throws IOException {
|
||||
|
||||
// NOTE: 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 TermVectorsWriter. This is
|
||||
// actually OK (unlike in the stored fields case)
|
||||
// because, although FieldInfos.hasVectors() will return
|
||||
// true, the TermVectorsReader gracefully handles
|
||||
// non-existence of the term vectors files.
|
||||
state.hasVectors = hasVectors;
|
||||
|
||||
if (tvx != null) {
|
||||
// At least one doc in this run had term vectors enabled
|
||||
fill(state.numDocs);
|
||||
tvx.close();
|
||||
tvf.close();
|
||||
tvd.close();
|
||||
tvx = tvd = tvf = null;
|
||||
assert state.segmentName != null;
|
||||
String idxName = IndexFileNames.segmentFileName(state.segmentName, "", IndexFileNames.VECTORS_INDEX_EXTENSION);
|
||||
String fldName = IndexFileNames.segmentFileName(state.segmentName, "", IndexFileNames.VECTORS_FIELDS_EXTENSION);
|
||||
String docName = IndexFileNames.segmentFileName(state.segmentName, "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION);
|
||||
|
||||
if (state.numDocsInStore > 0)
|
||||
// In case there are some final documents that we
|
||||
// didn't see (because they hit a non-aborting exception):
|
||||
fill(state.numDocsInStore - docWriter.getDocStoreOffset());
|
||||
if (4 + ((long) state.numDocs) * 16 != state.directory.fileLength(idxName)) {
|
||||
throw new RuntimeException("after flush: tvx size mismatch: " + state.numDocs + " docs vs " + state.directory.fileLength(idxName) + " length in bytes of " + idxName + " file exists?=" + state.directory.fileExists(idxName));
|
||||
}
|
||||
|
||||
tvx.flush();
|
||||
tvd.flush();
|
||||
tvf.flush();
|
||||
state.flushedFiles.add(idxName);
|
||||
state.flushedFiles.add(fldName);
|
||||
state.flushedFiles.add(docName);
|
||||
|
||||
lastDocID = 0;
|
||||
state.hasVectors = hasVectors;
|
||||
hasVectors = false;
|
||||
}
|
||||
|
||||
for (Map.Entry<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> entry : threadsAndFields.entrySet()) {
|
||||
|
@ -84,37 +87,6 @@ final class TermVectorsTermsWriter extends TermsHashConsumer {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
synchronized void closeDocStore(final SegmentWriteState state) throws IOException {
|
||||
if (tvx != null) {
|
||||
// At least one doc in this run had term vectors
|
||||
// enabled
|
||||
fill(state.numDocsInStore - docWriter.getDocStoreOffset());
|
||||
tvx.close();
|
||||
tvf.close();
|
||||
tvd.close();
|
||||
tvx = null;
|
||||
assert state.docStoreSegmentName != null;
|
||||
String idxName = IndexFileNames.segmentFileName(state.docStoreSegmentName, "", IndexFileNames.VECTORS_INDEX_EXTENSION);
|
||||
if (4+((long) state.numDocsInStore)*16 != state.directory.fileLength(idxName))
|
||||
throw new RuntimeException("after flush: tvx size mismatch: " + state.numDocsInStore + " docs vs " + state.directory.fileLength(idxName) + " length in bytes of " + idxName + " file exists?=" + state.directory.fileExists(idxName));
|
||||
|
||||
String fldName = IndexFileNames.segmentFileName(state.docStoreSegmentName, "", IndexFileNames.VECTORS_FIELDS_EXTENSION);
|
||||
String docName = IndexFileNames.segmentFileName(state.docStoreSegmentName, "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION);
|
||||
state.flushedFiles.add(idxName);
|
||||
state.flushedFiles.add(fldName);
|
||||
state.flushedFiles.add(docName);
|
||||
|
||||
docWriter.removeOpenFile(idxName);
|
||||
docWriter.removeOpenFile(fldName);
|
||||
docWriter.removeOpenFile(docName);
|
||||
|
||||
lastDocID = 0;
|
||||
state.hasVectors = hasVectors;
|
||||
hasVectors = false;
|
||||
}
|
||||
}
|
||||
|
||||
int allocCount;
|
||||
|
||||
synchronized PerDoc getPerDoc() {
|
||||
|
@ -128,18 +100,17 @@ final class TermVectorsTermsWriter extends TermsHashConsumer {
|
|||
docFreeList = new PerDoc[ArrayUtil.oversize(allocCount, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
|
||||
}
|
||||
return new PerDoc();
|
||||
} else
|
||||
} else {
|
||||
return docFreeList[--freeCount];
|
||||
}
|
||||
}
|
||||
|
||||
/** Fills in no-term-vectors for all docs we haven't seen
|
||||
* since the last doc that had term vectors. */
|
||||
void fill(int docID) throws IOException {
|
||||
final int docStoreOffset = docWriter.getDocStoreOffset();
|
||||
final int end = docID+docStoreOffset;
|
||||
if (lastDocID < end) {
|
||||
if (lastDocID < docID) {
|
||||
final long tvfPosition = tvf.getFilePointer();
|
||||
while(lastDocID < end) {
|
||||
while(lastDocID < docID) {
|
||||
tvx.writeLong(tvd.getFilePointer());
|
||||
tvd.writeVInt(0);
|
||||
tvx.writeLong(tvfPosition);
|
||||
|
@ -151,31 +122,19 @@ final class TermVectorsTermsWriter extends TermsHashConsumer {
|
|||
synchronized void initTermVectorsWriter() throws IOException {
|
||||
if (tvx == null) {
|
||||
|
||||
final String docStoreSegment = docWriter.getDocStoreSegment();
|
||||
|
||||
if (docStoreSegment == null)
|
||||
return;
|
||||
|
||||
// If we hit an exception while init'ing the term
|
||||
// vector output files, we must abort this segment
|
||||
// because those files will be in an unknown
|
||||
// state:
|
||||
String idxName = IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.VECTORS_INDEX_EXTENSION);
|
||||
String docName = IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION);
|
||||
String fldName = IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.VECTORS_FIELDS_EXTENSION);
|
||||
hasVectors = true;
|
||||
tvx = docWriter.directory.createOutput(idxName);
|
||||
tvd = docWriter.directory.createOutput(docName);
|
||||
tvf = docWriter.directory.createOutput(fldName);
|
||||
tvx = docWriter.directory.createOutput(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_INDEX_EXTENSION));
|
||||
tvd = docWriter.directory.createOutput(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION));
|
||||
tvf = docWriter.directory.createOutput(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_FIELDS_EXTENSION));
|
||||
|
||||
tvx.writeInt(TermVectorsReader.FORMAT_CURRENT);
|
||||
tvd.writeInt(TermVectorsReader.FORMAT_CURRENT);
|
||||
tvf.writeInt(TermVectorsReader.FORMAT_CURRENT);
|
||||
|
||||
docWriter.addOpenFile(idxName);
|
||||
docWriter.addOpenFile(fldName);
|
||||
docWriter.addOpenFile(docName);
|
||||
|
||||
lastDocID = 0;
|
||||
}
|
||||
}
|
||||
|
@ -193,8 +152,9 @@ final class TermVectorsTermsWriter extends TermsHashConsumer {
|
|||
tvx.writeLong(tvf.getFilePointer());
|
||||
tvd.writeVInt(perDoc.numVectorFields);
|
||||
if (perDoc.numVectorFields > 0) {
|
||||
for(int i=0;i<perDoc.numVectorFields;i++)
|
||||
for(int i=0;i<perDoc.numVectorFields;i++) {
|
||||
tvd.writeVInt(perDoc.fieldNumbers[i]);
|
||||
}
|
||||
assert 0 == perDoc.fieldPointers[0];
|
||||
long lastPos = perDoc.fieldPointers[0];
|
||||
for(int i=1;i<perDoc.numVectorFields;i++) {
|
||||
|
@ -206,7 +166,7 @@ final class TermVectorsTermsWriter extends TermsHashConsumer {
|
|||
perDoc.numVectorFields = 0;
|
||||
}
|
||||
|
||||
assert lastDocID == perDoc.docID + docWriter.getDocStoreOffset();
|
||||
assert lastDocID == perDoc.docID;
|
||||
|
||||
lastDocID++;
|
||||
|
||||
|
@ -215,36 +175,26 @@ final class TermVectorsTermsWriter extends TermsHashConsumer {
|
|||
assert docWriter.writer.testPoint("TermVectorsTermsWriter.finishDocument end");
|
||||
}
|
||||
|
||||
public boolean freeRAM() {
|
||||
// We don't hold any state beyond one doc, so we don't
|
||||
// free persistent RAM here
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void abort() {
|
||||
hasVectors = false;
|
||||
if (tvx != null) {
|
||||
try {
|
||||
tvx.close();
|
||||
} catch (Throwable t) {
|
||||
IOUtils.closeSafely(tvx, tvd, tvf);
|
||||
} catch (IOException ignored) {
|
||||
}
|
||||
tvx = null;
|
||||
}
|
||||
if (tvd != null) {
|
||||
try {
|
||||
tvd.close();
|
||||
} catch (Throwable t) {
|
||||
docWriter.directory.deleteFile(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_INDEX_EXTENSION));
|
||||
} catch (IOException ignored) {
|
||||
}
|
||||
tvd = null;
|
||||
}
|
||||
if (tvf != null) {
|
||||
try {
|
||||
tvf.close();
|
||||
} catch (Throwable t) {
|
||||
docWriter.directory.deleteFile(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION));
|
||||
} catch (IOException ignored) {
|
||||
}
|
||||
tvf = null;
|
||||
try {
|
||||
docWriter.directory.deleteFile(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_FIELDS_EXTENSION));
|
||||
} catch (IOException ignored) {
|
||||
}
|
||||
tvx = tvd = tvf = null;
|
||||
lastDocID = 0;
|
||||
}
|
||||
|
||||
|
|
|
@ -69,13 +69,6 @@ final class TermsHash extends InvertedDocConsumer {
|
|||
nextTermsHash.abort();
|
||||
}
|
||||
|
||||
@Override
|
||||
synchronized void closeDocStore(SegmentWriteState state) throws IOException {
|
||||
consumer.closeDocStore(state);
|
||||
if (nextTermsHash != null)
|
||||
nextTermsHash.closeDocStore(state);
|
||||
}
|
||||
|
||||
@Override
|
||||
synchronized void flush(Map<InvertedDocConsumerPerThread,Collection<InvertedDocConsumerPerField>> threadsAndFields, final SegmentWriteState state) throws IOException {
|
||||
Map<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> childThreadsAndFields = new HashMap<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>>();
|
||||
|
|
|
@ -25,7 +25,6 @@ abstract class TermsHashConsumer {
|
|||
abstract TermsHashConsumerPerThread addThread(TermsHashPerThread perThread);
|
||||
abstract void flush(Map<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> threadsAndFields, final SegmentWriteState state) throws IOException;
|
||||
abstract void abort();
|
||||
abstract void closeDocStore(SegmentWriteState state) throws IOException;
|
||||
|
||||
FieldInfos fieldInfos;
|
||||
|
||||
|
|
|
@ -64,4 +64,27 @@ public final class IOUtils {
|
|||
else if (firstIOE != null)
|
||||
throw firstIOE;
|
||||
}
|
||||
|
||||
/**
|
||||
* <p>Closes all given <tt>Closeable</tt>s, suppressing all thrown exceptions. Some of the <tt>Closeable</tt>s
|
||||
* may be null, they are ignored. After everything is closed, method either throws the first of suppressed exceptions,
|
||||
* or completes normally.</p>
|
||||
* @param objects objects to call <tt>close()</tt> on
|
||||
*/
|
||||
public static void closeSafely(Closeable... objects) throws IOException {
|
||||
IOException firstIOE = null;
|
||||
|
||||
for (Closeable object : objects) {
|
||||
try {
|
||||
if (object != null)
|
||||
object.close();
|
||||
} catch (IOException ioe) {
|
||||
if (firstIOE == null)
|
||||
firstIOE = ioe;
|
||||
}
|
||||
}
|
||||
|
||||
if (firstIOE != null)
|
||||
throw firstIOE;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,7 +17,6 @@ package org.apache.lucene;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.util.GregorianCalendar;
|
||||
import java.util.Random;
|
||||
import java.io.PrintWriter;
|
||||
import java.io.StringWriter;
|
||||
|
@ -77,7 +76,6 @@ public class TestSearch extends LuceneTestCase {
|
|||
IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer);
|
||||
LogMergePolicy lmp = (LogMergePolicy) conf.getMergePolicy();
|
||||
lmp.setUseCompoundFile(useCompoundFile);
|
||||
lmp.setUseCompoundDocStore(useCompoundFile);
|
||||
IndexWriter writer = new IndexWriter(directory, conf);
|
||||
|
||||
String[] docs = {
|
||||
|
|
|
@ -82,7 +82,6 @@ public class TestSearchForDuplicates extends LuceneTestCase {
|
|||
IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer);
|
||||
LogMergePolicy lmp = (LogMergePolicy) conf.getMergePolicy();
|
||||
lmp.setUseCompoundFile(useCompoundFiles);
|
||||
lmp.setUseCompoundDocStore(useCompoundFiles);
|
||||
IndexWriter writer = new IndexWriter(directory, conf);
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: now build index");
|
||||
|
|
|
@ -570,7 +570,6 @@ public class TestAddIndexes extends LuceneTestCase {
|
|||
Directory dir = newDirectory();
|
||||
LogByteSizeMergePolicy lmp = new LogByteSizeMergePolicy();
|
||||
lmp.setUseCompoundFile(false);
|
||||
lmp.setUseCompoundDocStore(false);
|
||||
lmp.setMergeFactor(100);
|
||||
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(
|
||||
TEST_VERSION_CURRENT, new MockAnalyzer())
|
||||
|
@ -599,7 +598,6 @@ public class TestAddIndexes extends LuceneTestCase {
|
|||
lmp = new LogByteSizeMergePolicy();
|
||||
lmp.setMinMergeMB(0.0001);
|
||||
lmp.setUseCompoundFile(false);
|
||||
lmp.setUseCompoundDocStore(false);
|
||||
lmp.setMergeFactor(4);
|
||||
writer = new IndexWriter(dir2, newIndexWriterConfig(TEST_VERSION_CURRENT,
|
||||
new MockAnalyzer())
|
||||
|
|
|
@ -500,7 +500,6 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
|
|||
Directory dir = newFSDirectory(new File(dirName));
|
||||
IndexWriterConfig conf = new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer()).setMaxBufferedDocs(10);
|
||||
((LogMergePolicy) conf.getMergePolicy()).setUseCompoundFile(doCFS);
|
||||
((LogMergePolicy) conf.getMergePolicy()).setUseCompoundDocStore(doCFS);
|
||||
IndexWriter writer = new IndexWriter(dir, conf);
|
||||
|
||||
for(int i=0;i<35;i++) {
|
||||
|
@ -512,7 +511,6 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
|
|||
// open fresh writer so we get no prx file in the added segment
|
||||
conf = new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer()).setMaxBufferedDocs(10);
|
||||
((LogMergePolicy) conf.getMergePolicy()).setUseCompoundFile(doCFS);
|
||||
((LogMergePolicy) conf.getMergePolicy()).setUseCompoundDocStore(doCFS);
|
||||
writer = new IndexWriter(dir, conf);
|
||||
addNoProxDoc(writer);
|
||||
writer.close();
|
||||
|
|
|
@ -270,7 +270,7 @@ public class TestCodecs extends LuceneTestCase {
|
|||
|
||||
final Directory dir = newDirectory();
|
||||
this.write(fieldInfos, dir, fields);
|
||||
final SegmentInfo si = new SegmentInfo(SEGMENT, 10000, dir, false, -1, SEGMENT, false, true, SegmentCodecs.build(fieldInfos, CodecProvider.getDefault()), fieldInfos.hasVectors());
|
||||
final SegmentInfo si = new SegmentInfo(SEGMENT, 10000, dir, false, true, SegmentCodecs.build(fieldInfos, CodecProvider.getDefault()), fieldInfos.hasVectors());
|
||||
si.setHasProx(false);
|
||||
|
||||
final FieldsProducer reader = si.getSegmentCodecs().codec().fieldsProducer(new SegmentReadState(dir, si, fieldInfos, 64, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR));
|
||||
|
@ -318,7 +318,7 @@ public class TestCodecs extends LuceneTestCase {
|
|||
final Directory dir = newDirectory();
|
||||
|
||||
this.write(fieldInfos, dir, fields);
|
||||
final SegmentInfo si = new SegmentInfo(SEGMENT, 10000, dir, false, -1, SEGMENT, false, true, SegmentCodecs.build(fieldInfos, CodecProvider.getDefault()), fieldInfos.hasVectors());
|
||||
final SegmentInfo si = new SegmentInfo(SEGMENT, 10000, dir, false, true, SegmentCodecs.build(fieldInfos, CodecProvider.getDefault()), fieldInfos.hasVectors());
|
||||
|
||||
final FieldsProducer terms = si.getSegmentCodecs().codec().fieldsProducer(new SegmentReadState(dir, si, fieldInfos, 1024, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR));
|
||||
|
||||
|
@ -607,7 +607,7 @@ public class TestCodecs extends LuceneTestCase {
|
|||
|
||||
final int termIndexInterval = this.nextInt(13, 27);
|
||||
final SegmentCodecs codecInfo = SegmentCodecs.build(fieldInfos, CodecProvider.getDefault());
|
||||
final SegmentWriteState state = new SegmentWriteState(null, dir, SEGMENT, fieldInfos, null, 10000, 10000, termIndexInterval, codecInfo);
|
||||
final SegmentWriteState state = new SegmentWriteState(null, dir, SEGMENT, fieldInfos, 10000, termIndexInterval, codecInfo);
|
||||
|
||||
final FieldsConsumer consumer = state.segmentCodecs.codec().fieldsConsumer(state);
|
||||
Arrays.sort(fields);
|
||||
|
|
|
@ -96,7 +96,7 @@ public class TestConcurrentMergeScheduler extends LuceneTestCase {
|
|||
writer.addDocument(doc);
|
||||
failure.setDoFail();
|
||||
try {
|
||||
writer.flush(true, false, true);
|
||||
writer.flush(true, true);
|
||||
if (failure.hitExc) {
|
||||
fail("failed to hit IOException");
|
||||
}
|
||||
|
|
|
@ -195,16 +195,13 @@ public class TestDeletionPolicy extends LuceneTestCase {
|
|||
|
||||
final double SECONDS = 2.0;
|
||||
|
||||
boolean useCompoundFile = true;
|
||||
|
||||
Directory dir = newDirectory();
|
||||
ExpirationTimeDeletionPolicy policy = new ExpirationTimeDeletionPolicy(dir, SECONDS);
|
||||
IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT,
|
||||
new MockAnalyzer())
|
||||
.setIndexDeletionPolicy(policy);
|
||||
LogMergePolicy lmp = (LogMergePolicy) conf.getMergePolicy();
|
||||
lmp.setUseCompoundFile(useCompoundFile);
|
||||
lmp.setUseCompoundDocStore(useCompoundFile);
|
||||
lmp.setUseCompoundFile(true);
|
||||
IndexWriter writer = new IndexWriter(dir, conf);
|
||||
writer.close();
|
||||
|
||||
|
@ -219,8 +216,7 @@ public class TestDeletionPolicy extends LuceneTestCase {
|
|||
new MockAnalyzer()).setOpenMode(
|
||||
OpenMode.APPEND).setIndexDeletionPolicy(policy);
|
||||
lmp = (LogMergePolicy) conf.getMergePolicy();
|
||||
lmp.setUseCompoundFile(useCompoundFile);
|
||||
lmp.setUseCompoundDocStore(useCompoundFile);
|
||||
lmp.setUseCompoundFile(true);
|
||||
writer = new IndexWriter(dir, conf);
|
||||
for(int j=0;j<17;j++) {
|
||||
addDoc(writer);
|
||||
|
@ -298,7 +294,6 @@ public class TestDeletionPolicy extends LuceneTestCase {
|
|||
.setMergeScheduler(new SerialMergeScheduler());
|
||||
LogMergePolicy lmp = (LogMergePolicy) conf.getMergePolicy();
|
||||
lmp.setUseCompoundFile(useCompoundFile);
|
||||
lmp.setUseCompoundDocStore(useCompoundFile);
|
||||
lmp.setMergeFactor(10);
|
||||
IndexWriter writer = new IndexWriter(dir, conf);
|
||||
for(int i=0;i<107;i++) {
|
||||
|
@ -311,7 +306,6 @@ public class TestDeletionPolicy extends LuceneTestCase {
|
|||
OpenMode.APPEND).setIndexDeletionPolicy(policy);
|
||||
lmp = (LogMergePolicy) conf.getMergePolicy();
|
||||
lmp.setUseCompoundFile(useCompoundFile);
|
||||
lmp.setUseCompoundDocStore(useCompoundFile);
|
||||
writer = new IndexWriter(dir, conf);
|
||||
writer.optimize();
|
||||
writer.close();
|
||||
|
@ -488,7 +482,6 @@ public class TestDeletionPolicy extends LuceneTestCase {
|
|||
.setMaxBufferedDocs(10);
|
||||
LogMergePolicy lmp = (LogMergePolicy) conf.getMergePolicy();
|
||||
lmp.setUseCompoundFile(useCompoundFile);
|
||||
lmp.setUseCompoundDocStore(useCompoundFile);
|
||||
IndexWriter writer = new IndexWriter(dir, conf);
|
||||
for(int i=0;i<107;i++) {
|
||||
addDoc(writer);
|
||||
|
@ -499,7 +492,6 @@ public class TestDeletionPolicy extends LuceneTestCase {
|
|||
.setOpenMode(OpenMode.APPEND).setIndexDeletionPolicy(policy);
|
||||
lmp = (LogMergePolicy) conf.getMergePolicy();
|
||||
lmp.setUseCompoundFile(useCompoundFile);
|
||||
lmp.setUseCompoundDocStore(useCompoundFile);
|
||||
writer = new IndexWriter(dir, conf);
|
||||
writer.optimize();
|
||||
writer.close();
|
||||
|
@ -539,7 +531,6 @@ public class TestDeletionPolicy extends LuceneTestCase {
|
|||
.setMaxBufferedDocs(10);
|
||||
LogMergePolicy lmp = (LogMergePolicy) conf.getMergePolicy();
|
||||
lmp.setUseCompoundFile(useCompoundFile);
|
||||
lmp.setUseCompoundDocStore(useCompoundFile);
|
||||
IndexWriter writer = new IndexWriter(dir, conf);
|
||||
for(int i=0;i<17;i++) {
|
||||
addDoc(writer);
|
||||
|
@ -597,7 +588,6 @@ public class TestDeletionPolicy extends LuceneTestCase {
|
|||
.setOpenMode(OpenMode.CREATE).setIndexDeletionPolicy(policy);
|
||||
LogMergePolicy lmp = (LogMergePolicy) conf.getMergePolicy();
|
||||
lmp.setUseCompoundFile(useCompoundFile);
|
||||
lmp.setUseCompoundDocStore(useCompoundFile);
|
||||
IndexWriter writer = new IndexWriter(dir, conf);
|
||||
writer.close();
|
||||
Term searchTerm = new Term("content", "aaa");
|
||||
|
@ -609,7 +599,6 @@ public class TestDeletionPolicy extends LuceneTestCase {
|
|||
.setOpenMode(OpenMode.APPEND).setIndexDeletionPolicy(policy);
|
||||
lmp = (LogMergePolicy) conf.getMergePolicy();
|
||||
lmp.setUseCompoundFile(useCompoundFile);
|
||||
lmp.setUseCompoundDocStore(useCompoundFile);
|
||||
writer = new IndexWriter(dir, conf);
|
||||
for(int j=0;j<17;j++) {
|
||||
addDoc(writer);
|
||||
|
@ -630,7 +619,6 @@ public class TestDeletionPolicy extends LuceneTestCase {
|
|||
.setOpenMode(OpenMode.APPEND).setIndexDeletionPolicy(policy);
|
||||
lmp = (LogMergePolicy) conf.getMergePolicy();
|
||||
lmp.setUseCompoundFile(useCompoundFile);
|
||||
lmp.setUseCompoundDocStore(useCompoundFile);
|
||||
writer = new IndexWriter(dir, conf);
|
||||
writer.optimize();
|
||||
// this is a commit
|
||||
|
@ -706,7 +694,6 @@ public class TestDeletionPolicy extends LuceneTestCase {
|
|||
.setMaxBufferedDocs(10);
|
||||
LogMergePolicy lmp = (LogMergePolicy) conf.getMergePolicy();
|
||||
lmp.setUseCompoundFile(useCompoundFile);
|
||||
lmp.setUseCompoundDocStore(useCompoundFile);
|
||||
IndexWriter writer = new IndexWriter(dir, conf);
|
||||
writer.close();
|
||||
Term searchTerm = new Term("content", "aaa");
|
||||
|
@ -720,7 +707,6 @@ public class TestDeletionPolicy extends LuceneTestCase {
|
|||
.setMaxBufferedDocs(10);
|
||||
lmp = (LogMergePolicy) conf.getMergePolicy();
|
||||
lmp.setUseCompoundFile(useCompoundFile);
|
||||
lmp.setUseCompoundDocStore(useCompoundFile);
|
||||
writer = new IndexWriter(dir, conf);
|
||||
for(int j=0;j<17;j++) {
|
||||
addDoc(writer);
|
||||
|
|
|
@ -201,7 +201,7 @@ public class TestDoc extends LuceneTestCase {
|
|||
r2.close();
|
||||
|
||||
final SegmentInfo info = new SegmentInfo(merged, si1.docCount + si2.docCount, si1.dir,
|
||||
useCompoundFile, -1, null, false, merger.fieldInfos().hasProx(), merger.getSegmentCodecs(),
|
||||
useCompoundFile, merger.fieldInfos().hasProx(), merger.getSegmentCodecs(),
|
||||
merger.fieldInfos().hasVectors());
|
||||
|
||||
if (useCompoundFile) {
|
||||
|
|
|
@ -53,7 +53,6 @@ public class TestFieldsReader extends LuceneTestCase {
|
|||
dir = newDirectory();
|
||||
IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer());
|
||||
((LogMergePolicy) conf.getMergePolicy()).setUseCompoundFile(false);
|
||||
((LogMergePolicy) conf.getMergePolicy()).setUseCompoundDocStore(false);
|
||||
IndexWriter writer = new IndexWriter(dir, conf);
|
||||
writer.addDocument(testDoc);
|
||||
writer.close();
|
||||
|
|
|
@ -56,7 +56,6 @@ public class TestIndexFileDeleter extends LuceneTestCase {
|
|||
addDoc(writer, i);
|
||||
}
|
||||
mergePolicy.setUseCompoundFile(false);
|
||||
mergePolicy.setUseCompoundDocStore(false);
|
||||
for(;i<45;i++) {
|
||||
addDoc(writer, i);
|
||||
}
|
||||
|
|
|
@ -240,7 +240,6 @@ public class TestIndexReaderCloneNorms extends LuceneTestCase {
|
|||
LogMergePolicy lmp = (LogMergePolicy) iw.getConfig().getMergePolicy();
|
||||
lmp.setMergeFactor(3);
|
||||
lmp.setUseCompoundFile(true);
|
||||
lmp.setUseCompoundDocStore(true);
|
||||
iw.close();
|
||||
}
|
||||
|
||||
|
@ -293,7 +292,6 @@ public class TestIndexReaderCloneNorms extends LuceneTestCase {
|
|||
LogMergePolicy lmp = (LogMergePolicy) conf.getMergePolicy();
|
||||
lmp.setMergeFactor(3);
|
||||
lmp.setUseCompoundFile(compound);
|
||||
lmp.setUseCompoundDocStore(compound);
|
||||
IndexWriter iw = new IndexWriter(dir, conf);
|
||||
for (int i = 0; i < ndocs; i++) {
|
||||
iw.addDocument(newDoc());
|
||||
|
|
|
@ -157,7 +157,7 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
String[] startFiles = dir.listAll();
|
||||
SegmentInfos infos = new SegmentInfos();
|
||||
infos.read(dir);
|
||||
new IndexFileDeleter(dir, new KeepOnlyLastCommitDeletionPolicy(), infos, null, null, CodecProvider.getDefault());
|
||||
new IndexFileDeleter(dir, new KeepOnlyLastCommitDeletionPolicy(), infos, null, CodecProvider.getDefault());
|
||||
String[] endFiles = dir.listAll();
|
||||
|
||||
Arrays.sort(startFiles);
|
||||
|
@ -866,7 +866,7 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
doc.add(newField("field", "aaa", Store.YES, Index.ANALYZED, TermVector.WITH_POSITIONS_OFFSETS));
|
||||
for(int i=0;i<19;i++)
|
||||
writer.addDocument(doc);
|
||||
writer.flush(false, true, true);
|
||||
writer.flush(false, true);
|
||||
writer.close();
|
||||
SegmentInfos sis = new SegmentInfos();
|
||||
sis.read(dir);
|
||||
|
@ -1039,7 +1039,6 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
LogMergePolicy lmp = (LogMergePolicy) writer.getConfig().getMergePolicy();
|
||||
lmp.setMergeFactor(2);
|
||||
lmp.setUseCompoundFile(false);
|
||||
lmp.setUseCompoundDocStore(false);
|
||||
Document doc = new Document();
|
||||
String contents = "aa bb cc dd ee ff gg hh ii jj kk";
|
||||
|
||||
|
@ -1075,7 +1074,6 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
writer = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer()));
|
||||
LogMergePolicy lmp2 = (LogMergePolicy) writer.getConfig().getMergePolicy();
|
||||
lmp2.setUseCompoundFile(false);
|
||||
lmp2.setUseCompoundDocStore(false);
|
||||
writer.optimize();
|
||||
writer.close();
|
||||
}
|
||||
|
@ -2257,7 +2255,6 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
TEST_VERSION_CURRENT, new MockAnalyzer()));
|
||||
LogMergePolicy lmp = (LogMergePolicy) writer.getConfig().getMergePolicy();
|
||||
lmp.setUseCompoundFile(false);
|
||||
lmp.setUseCompoundDocStore(false);
|
||||
ByteArrayOutputStream bos = new ByteArrayOutputStream(1024);
|
||||
writer.setInfoStream(new PrintStream(bos));
|
||||
writer.addDocument(new Document());
|
||||
|
|
|
@ -689,7 +689,6 @@ public class TestIndexWriterDelete extends LuceneTestCase {
|
|||
|
||||
LogMergePolicy lmp = (LogMergePolicy) modifier.getConfig().getMergePolicy();
|
||||
lmp.setUseCompoundFile(true);
|
||||
lmp.setUseCompoundDocStore(true);
|
||||
|
||||
dir.failOn(failure.reset());
|
||||
|
||||
|
|
|
@ -237,7 +237,7 @@ public class TestIndexWriterReader extends LuceneTestCase {
|
|||
writer.setInfoStream(infoStream);
|
||||
// create the index
|
||||
createIndexNoClose(!optimize, "index1", writer);
|
||||
writer.flush(false, true, true);
|
||||
writer.flush(false, true);
|
||||
|
||||
// create a 2nd index
|
||||
Directory dir2 = newDirectory();
|
||||
|
@ -317,7 +317,7 @@ public class TestIndexWriterReader extends LuceneTestCase {
|
|||
writer.setInfoStream(infoStream);
|
||||
// create the index
|
||||
createIndexNoClose(!optimize, "index1", writer);
|
||||
writer.flush(false, true, true);
|
||||
writer.flush(false, true);
|
||||
// get a reader
|
||||
IndexReader r1 = writer.getReader();
|
||||
|
||||
|
@ -524,7 +524,7 @@ public class TestIndexWriterReader extends LuceneTestCase {
|
|||
IndexReader r1 = writer.getReader();
|
||||
assertEquals(0, r1.maxDoc());
|
||||
createIndexNoClose(false, "index1", writer);
|
||||
writer.flush(!optimize, true, true);
|
||||
writer.flush(!optimize, true);
|
||||
|
||||
IndexReader iwr1 = writer.getReader();
|
||||
assertEquals(100, iwr1.maxDoc());
|
||||
|
@ -536,7 +536,7 @@ public class TestIndexWriterReader extends LuceneTestCase {
|
|||
Document d = createDocument(x, "index1", 5);
|
||||
writer.addDocument(d);
|
||||
}
|
||||
writer.flush(false, true, true);
|
||||
writer.flush(false, true);
|
||||
// verify the reader was reopened internally
|
||||
IndexReader iwr2 = writer.getReader();
|
||||
assertTrue(iwr2 != r1);
|
||||
|
|
|
@ -358,47 +358,6 @@ public class TestIndexWriterWithThreads extends LuceneTestCase {
|
|||
_testMultipleThreadsFailure(new FailOnlyOnAbortOrFlush(true));
|
||||
}
|
||||
|
||||
// Throws IOException during DocumentsWriter.closeDocStore
|
||||
private static class FailOnlyInCloseDocStore extends MockDirectoryWrapper.Failure {
|
||||
private boolean onlyOnce;
|
||||
public FailOnlyInCloseDocStore(boolean onlyOnce) {
|
||||
this.onlyOnce = onlyOnce;
|
||||
}
|
||||
@Override
|
||||
public void eval(MockDirectoryWrapper dir) throws IOException {
|
||||
if (doFail) {
|
||||
StackTraceElement[] trace = new Exception().getStackTrace();
|
||||
for (int i = 0; i < trace.length; i++) {
|
||||
if ("closeDocStore".equals(trace[i].getMethodName())) {
|
||||
if (onlyOnce)
|
||||
doFail = false;
|
||||
throw new IOException("now failing on purpose");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// LUCENE-1130: test IOException in closeDocStore
|
||||
public void testIOExceptionDuringCloseDocStore() throws IOException {
|
||||
_testSingleThreadFailure(new FailOnlyInCloseDocStore(false));
|
||||
}
|
||||
|
||||
// LUCENE-1130: test IOException in closeDocStore
|
||||
public void testIOExceptionDuringCloseDocStoreOnlyOnce() throws IOException {
|
||||
_testSingleThreadFailure(new FailOnlyInCloseDocStore(true));
|
||||
}
|
||||
|
||||
// LUCENE-1130: test IOException in closeDocStore, with threads
|
||||
public void testIOExceptionDuringCloseDocStoreWithThreads() throws Exception {
|
||||
_testMultipleThreadsFailure(new FailOnlyInCloseDocStore(false));
|
||||
}
|
||||
|
||||
// LUCENE-1130: test IOException in closeDocStore, with threads
|
||||
public void testIOExceptionDuringCloseDocStoreWithThreadsOnlyOnce() throws Exception {
|
||||
_testMultipleThreadsFailure(new FailOnlyInCloseDocStore(true));
|
||||
}
|
||||
|
||||
// Throws IOException during DocumentsWriter.writeSegment
|
||||
private static class FailOnlyInWriteSegment extends MockDirectoryWrapper.Failure {
|
||||
private boolean onlyOnce;
|
||||
|
|
|
@ -66,7 +66,6 @@ public class TestLazyBug extends LuceneTestCase {
|
|||
TEST_VERSION_CURRENT, new MockAnalyzer()));
|
||||
LogMergePolicy lmp = (LogMergePolicy) writer.getConfig().getMergePolicy();
|
||||
lmp.setUseCompoundFile(false);
|
||||
lmp.setUseCompoundDocStore(false);
|
||||
|
||||
for (int d = 1; d <= NUM_DOCS; d++) {
|
||||
Document doc = new Document();
|
||||
|
|
|
@ -35,16 +35,13 @@ public class TestNoMergePolicy extends LuceneTestCase {
|
|||
assertNull(mp.findMerges(null));
|
||||
assertNull(mp.findMergesForOptimize(null, 0, null));
|
||||
assertNull(mp.findMergesToExpungeDeletes(null));
|
||||
assertFalse(mp.useCompoundDocStore(null));
|
||||
assertFalse(mp.useCompoundFile(null, null));
|
||||
mp.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCompoundFiles() throws Exception {
|
||||
assertFalse(NoMergePolicy.NO_COMPOUND_FILES.useCompoundDocStore(null));
|
||||
assertFalse(NoMergePolicy.NO_COMPOUND_FILES.useCompoundFile(null, null));
|
||||
assertTrue(NoMergePolicy.COMPOUND_FILES.useCompoundDocStore(null));
|
||||
assertTrue(NoMergePolicy.COMPOUND_FILES.useCompoundFile(null, null));
|
||||
}
|
||||
|
||||
|
|
|
@ -154,7 +154,6 @@ public class TestNorms extends LuceneTestCase {
|
|||
LogMergePolicy lmp = (LogMergePolicy) iw.getConfig().getMergePolicy();
|
||||
lmp.setMergeFactor(3);
|
||||
lmp.setUseCompoundFile(true);
|
||||
lmp.setUseCompoundDocStore(true);
|
||||
iw.close();
|
||||
}
|
||||
|
||||
|
@ -199,7 +198,6 @@ public class TestNorms extends LuceneTestCase {
|
|||
LogMergePolicy lmp = (LogMergePolicy) iw.getConfig().getMergePolicy();
|
||||
lmp.setMergeFactor(3);
|
||||
lmp.setUseCompoundFile(compound);
|
||||
lmp.setUseCompoundDocStore(compound);
|
||||
for (int i = 0; i < ndocs; i++) {
|
||||
iw.addDocument(newDoc());
|
||||
}
|
||||
|
|
|
@ -219,7 +219,6 @@ public class TestOmitTf extends LuceneTestCase {
|
|||
LogMergePolicy lmp = (LogMergePolicy) writer.getConfig().getMergePolicy();
|
||||
lmp.setMergeFactor(2);
|
||||
lmp.setUseCompoundFile(false);
|
||||
lmp.setUseCompoundDocStore(false);
|
||||
Document d = new Document();
|
||||
|
||||
Field f1 = newField("f1", "This field has term freqs", Field.Store.NO, Field.Index.ANALYZED);
|
||||
|
|
|
@ -137,7 +137,6 @@ public class TestPerFieldCodecSupport extends LuceneTestCase {
|
|||
iwconf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer())
|
||||
.setOpenMode(OpenMode.APPEND).setCodecProvider(provider);
|
||||
((LogMergePolicy) iwconf.getMergePolicy()).setUseCompoundFile(false);
|
||||
((LogMergePolicy) iwconf.getMergePolicy()).setUseCompoundDocStore(false);
|
||||
((LogMergePolicy) iwconf.getMergePolicy()).setMergeFactor(10);
|
||||
iwconf.setMaxBufferedDocs(IndexWriterConfig.DISABLE_AUTO_FLUSH);
|
||||
|
||||
|
|
|
@ -24,7 +24,6 @@ import java.util.Random;
|
|||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.index.TermsEnum.SeekStatus;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.MockDirectoryWrapper;
|
||||
|
@ -73,7 +72,7 @@ public class TestPerSegmentDeletes extends LuceneTestCase {
|
|||
|
||||
// flushing without applying deletes means
|
||||
// there will still be deletes in the segment infos
|
||||
writer.flush(false, false, false);
|
||||
writer.flush(false, false);
|
||||
assertTrue(writer.bufferedDeletes.any());
|
||||
|
||||
// get reader flushes pending deletes
|
||||
|
@ -86,7 +85,7 @@ public class TestPerSegmentDeletes extends LuceneTestCase {
|
|||
// merge segments 0 and 1
|
||||
// which should apply the delete id:2
|
||||
writer.deleteDocuments(new Term("id", "2"));
|
||||
writer.flush(false, false, false);
|
||||
writer.flush(false, false);
|
||||
fsmp.doMerge = true;
|
||||
fsmp.start = 0;
|
||||
fsmp.length = 2;
|
||||
|
@ -179,12 +178,12 @@ public class TestPerSegmentDeletes extends LuceneTestCase {
|
|||
writer.addDocument(TestIndexWriterReader.createDocument(x, "5", 2));
|
||||
//System.out.println("numRamDocs(" + x + ")" + writer.numRamDocs());
|
||||
}
|
||||
writer.flush(false, true, false);
|
||||
writer.flush(false, false);
|
||||
for (int x = 25; x < 30; x++) {
|
||||
writer.addDocument(TestIndexWriterReader.createDocument(x, "5", 2));
|
||||
//System.out.println("numRamDocs(" + x + ")" + writer.numRamDocs());
|
||||
}
|
||||
writer.flush(false, true, false);
|
||||
writer.flush(false, false);
|
||||
|
||||
//System.out.println("infos3:"+writer.segmentInfos);
|
||||
|
||||
|
@ -286,11 +285,6 @@ public class TestPerSegmentDeletes extends LuceneTestCase {
|
|||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean useCompoundDocStore(SegmentInfos segments) {
|
||||
return useCompoundFile;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean useCompoundFile(SegmentInfos segments, SegmentInfo newSegment) {
|
||||
return useCompoundFile;
|
||||
|
|
|
@ -79,8 +79,7 @@ public class TestSegmentMerger extends LuceneTestCase {
|
|||
int docsMerged = merger.merge();
|
||||
assertTrue(docsMerged == 2);
|
||||
//Should be able to open a new SegmentReader against the new directory
|
||||
SegmentReader mergedReader = SegmentReader.get(false, mergedDir, new SegmentInfo(mergedSegment, docsMerged, mergedDir, false, -1,
|
||||
null, false, merger.fieldInfos().hasProx(),
|
||||
SegmentReader mergedReader = SegmentReader.get(false, mergedDir, new SegmentInfo(mergedSegment, docsMerged, mergedDir, false, merger.fieldInfos().hasProx(),
|
||||
merger.getSegmentCodecs(), merger.fieldInfos().hasVectors()),
|
||||
BufferedIndexInput.BUFFER_SIZE, true, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR);
|
||||
|
||||
|
|
|
@ -154,7 +154,6 @@ public class TestStressIndexing2 extends LuceneTestCase {
|
|||
w.commit();
|
||||
LogMergePolicy lmp = (LogMergePolicy) w.getConfig().getMergePolicy();
|
||||
lmp.setUseCompoundFile(false);
|
||||
lmp.setUseCompoundDocStore(false);
|
||||
lmp.setMergeFactor(mergeFactor);
|
||||
/***
|
||||
w.setMaxMergeDocs(Integer.MAX_VALUE);
|
||||
|
@ -211,7 +210,6 @@ public class TestStressIndexing2 extends LuceneTestCase {
|
|||
w.setInfoStream(VERBOSE ? System.out : null);
|
||||
LogMergePolicy lmp = (LogMergePolicy) w.getConfig().getMergePolicy();
|
||||
lmp.setUseCompoundFile(false);
|
||||
lmp.setUseCompoundDocStore(false);
|
||||
lmp.setMergeFactor(mergeFactor);
|
||||
|
||||
threads = new IndexingThread[nThreads];
|
||||
|
|
|
@ -688,7 +688,6 @@ public abstract class LuceneTestCase extends Assert {
|
|||
|
||||
public static LogMergePolicy newLogMergePolicy(Random r) {
|
||||
LogMergePolicy logmp = r.nextBoolean() ? new LogDocMergePolicy() : new LogByteSizeMergePolicy();
|
||||
logmp.setUseCompoundDocStore(r.nextBoolean());
|
||||
logmp.setUseCompoundFile(r.nextBoolean());
|
||||
logmp.setCalibrateSizeByDeletes(r.nextBoolean());
|
||||
if (r.nextInt(3) == 2) {
|
||||
|
@ -702,14 +701,12 @@ public abstract class LuceneTestCase extends Assert {
|
|||
public static LogMergePolicy newLogMergePolicy(boolean useCFS) {
|
||||
LogMergePolicy logmp = newLogMergePolicy();
|
||||
logmp.setUseCompoundFile(useCFS);
|
||||
logmp.setUseCompoundDocStore(useCFS);
|
||||
return logmp;
|
||||
}
|
||||
|
||||
public static LogMergePolicy newLogMergePolicy(boolean useCFS, int mergeFactor) {
|
||||
LogMergePolicy logmp = newLogMergePolicy();
|
||||
logmp.setUseCompoundFile(useCFS);
|
||||
logmp.setUseCompoundDocStore(useCFS);
|
||||
logmp.setMergeFactor(mergeFactor);
|
||||
return logmp;
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue