mirror of https://github.com/apache/lucene.git
LUCENE-3147: MockDirectoryWrapper should track open file handles of IndexOutput
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1128830 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
0771a503cb
commit
c5f5f67b25
|
@ -427,6 +427,11 @@ Bug fixes
|
|||
with more document deletions is requested before a reader with fewer
|
||||
deletions, provided they share some segments. (yonik)
|
||||
|
||||
* LUCENE-3147,LUCENE-3152: Fixed open file handles leaks in many places in the
|
||||
code. Now MockDirectoryWrapper (in test-framework) tracks all open files,
|
||||
including locks, and fails if the test fails to release all of them.
|
||||
(Mike McCandless, Robert Muir, Shai Erera, Simon Willnauer)
|
||||
|
||||
======================= Lucene 3.x (not yet released) ================
|
||||
|
||||
(No changes)
|
||||
|
|
|
@ -269,7 +269,7 @@ public class NRTCachingDirectory extends Directory {
|
|||
in = cache.openInput(fileName);
|
||||
in.copyBytes(out, in.length());
|
||||
} finally {
|
||||
IOUtils.closeSafely(in, out);
|
||||
IOUtils.closeSafely(false, in, out);
|
||||
}
|
||||
synchronized(this) {
|
||||
cache.deleteFile(fileName);
|
||||
|
|
|
@ -71,7 +71,15 @@ final class DocFieldProcessor extends DocConsumer {
|
|||
childFields.put(f.getFieldInfo(), f);
|
||||
}
|
||||
|
||||
fieldsWriter.flush(state);
|
||||
boolean success = false;
|
||||
try {
|
||||
fieldsWriter.flush(state);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
abort();
|
||||
}
|
||||
}
|
||||
consumer.flush(childFields, state);
|
||||
|
||||
// Important to save after asking consumer to flush so
|
||||
|
@ -84,19 +92,44 @@ final class DocFieldProcessor extends DocConsumer {
|
|||
|
||||
@Override
|
||||
public void abort() {
|
||||
for(int i=0;i<fieldHash.length;i++) {
|
||||
DocFieldProcessorPerField field = fieldHash[i];
|
||||
while(field != null) {
|
||||
Throwable th = null;
|
||||
|
||||
for (DocFieldProcessorPerField field : fieldHash) {
|
||||
while (field != null) {
|
||||
final DocFieldProcessorPerField next = field.next;
|
||||
field.abort();
|
||||
try {
|
||||
field.abort();
|
||||
} catch (Throwable t) {
|
||||
if (th == null) {
|
||||
th = t;
|
||||
}
|
||||
}
|
||||
field = next;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
try {
|
||||
fieldsWriter.abort();
|
||||
} finally {
|
||||
} catch (Throwable t) {
|
||||
if (th == null) {
|
||||
th = t;
|
||||
}
|
||||
}
|
||||
|
||||
try {
|
||||
consumer.abort();
|
||||
} catch (Throwable t) {
|
||||
if (th == null) {
|
||||
th = t;
|
||||
}
|
||||
}
|
||||
|
||||
// If any errors occured, throw it.
|
||||
if (th != null) {
|
||||
if (th instanceof RuntimeException) throw (RuntimeException) th;
|
||||
if (th instanceof Error) throw (Error) th;
|
||||
// defensive code - we should not hit unchecked exceptions
|
||||
throw new RuntimeException(th);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -87,6 +87,7 @@ final class DocInverter extends DocFieldConsumer {
|
|||
endConsumer.startDocument();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void finishDocument() throws IOException {
|
||||
// TODO: allow endConsumer.finishDocument to also return
|
||||
// a DocWriter
|
||||
|
|
|
@ -53,8 +53,11 @@ final class DocInverterPerField extends DocFieldConsumerPerField {
|
|||
|
||||
@Override
|
||||
void abort() {
|
||||
consumer.abort();
|
||||
endConsumer.abort();
|
||||
try {
|
||||
consumer.abort();
|
||||
} finally {
|
||||
endConsumer.abort();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -228,14 +228,19 @@ final class DocumentsWriter {
|
|||
}
|
||||
|
||||
final Iterator<ThreadState> threadsIterator = perThreadPool.getActivePerThreadsIterator();
|
||||
|
||||
while (threadsIterator.hasNext()) {
|
||||
ThreadState perThread = threadsIterator.next();
|
||||
final ThreadState perThread = threadsIterator.next();
|
||||
perThread.lock();
|
||||
try {
|
||||
if (perThread.isActive()) { // we might be closed
|
||||
perThread.perThread.abort();
|
||||
perThread.perThread.checkAndResetHasAborted();
|
||||
try {
|
||||
perThread.perThread.abort();
|
||||
} catch (IOException ex) {
|
||||
// continue
|
||||
} finally {
|
||||
perThread.perThread.checkAndResetHasAborted();
|
||||
flushControl.doOnAbort(perThread);
|
||||
}
|
||||
} else {
|
||||
assert closed;
|
||||
}
|
||||
|
@ -243,7 +248,6 @@ final class DocumentsWriter {
|
|||
perThread.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
success = true;
|
||||
} finally {
|
||||
if (infoStream != null) {
|
||||
|
|
|
@ -16,6 +16,7 @@ package org.apache.lucene.index;
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
|
@ -204,7 +205,7 @@ public final class DocumentsWriterFlushControl {
|
|||
} // don't assert on numDocs since we could hit an abort excp. while selecting that dwpt for flushing
|
||||
|
||||
}
|
||||
|
||||
|
||||
synchronized void doOnAbort(ThreadState state) {
|
||||
try {
|
||||
if (state.flushPending) {
|
||||
|
@ -449,10 +450,21 @@ public final class DocumentsWriterFlushControl {
|
|||
try {
|
||||
for (DocumentsWriterPerThread dwpt : flushQueue) {
|
||||
doAfterFlush(dwpt);
|
||||
try {
|
||||
dwpt.abort();
|
||||
} catch (IOException ex) {
|
||||
// continue
|
||||
}
|
||||
}
|
||||
for (BlockedFlush blockedFlush : blockedFlushes) {
|
||||
flushingWriters.put(blockedFlush.dwpt, Long.valueOf(blockedFlush.bytes));
|
||||
flushingWriters
|
||||
.put(blockedFlush.dwpt, Long.valueOf(blockedFlush.bytes));
|
||||
doAfterFlush(blockedFlush.dwpt);
|
||||
try {
|
||||
blockedFlush.dwpt.abort();
|
||||
} catch (IOException ex) {
|
||||
// continue
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
fullFlush = false;
|
||||
|
@ -511,4 +523,4 @@ public final class DocumentsWriterFlushControl {
|
|||
return stallControl.anyStalledThreads();
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
@ -177,7 +177,7 @@ public class DocumentsWriterPerThread {
|
|||
this.parent = parent;
|
||||
this.fieldInfos = fieldInfos;
|
||||
this.writer = parent.indexWriter;
|
||||
this.infoStream = parent.indexWriter.getInfoStream();
|
||||
this.infoStream = parent.infoStream;
|
||||
this.docState = new DocState(this);
|
||||
this.docState.similarityProvider = parent.indexWriter.getConfig()
|
||||
.getSimilarityProvider();
|
||||
|
@ -550,6 +550,7 @@ public class DocumentsWriterPerThread {
|
|||
super(blockSize);
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getByteBlock() {
|
||||
bytesUsed.addAndGet(blockSize);
|
||||
return new byte[blockSize];
|
||||
|
@ -562,7 +563,7 @@ public class DocumentsWriterPerThread {
|
|||
}
|
||||
}
|
||||
|
||||
};
|
||||
}
|
||||
|
||||
void setInfoStream(PrintStream infoStream) {
|
||||
this.infoStream = infoStream;
|
||||
|
|
|
@ -113,7 +113,7 @@ final class FieldsWriter {
|
|||
void close() throws IOException {
|
||||
if (directory != null) {
|
||||
try {
|
||||
IOUtils.closeSafely(fieldsStream, indexStream);
|
||||
IOUtils.closeSafely(false, fieldsStream, indexStream);
|
||||
} finally {
|
||||
fieldsStream = indexStream = null;
|
||||
}
|
||||
|
|
|
@ -22,6 +22,7 @@ import java.util.Collection;
|
|||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
// TODO FI: norms could actually be stored as doc store
|
||||
|
||||
|
@ -49,7 +50,7 @@ final class NormsWriter extends InvertedDocEndConsumer {
|
|||
|
||||
final String normsFileName = IndexFileNames.segmentFileName(state.segmentName, "", IndexFileNames.NORMS_EXTENSION);
|
||||
IndexOutput normsOut = state.directory.createOutput(normsFileName);
|
||||
|
||||
boolean success = false;
|
||||
try {
|
||||
normsOut.writeBytes(SegmentNorms.NORMS_HEADER, 0, SegmentNorms.NORMS_HEADER.length);
|
||||
|
||||
|
@ -84,9 +85,9 @@ final class NormsWriter extends InvertedDocEndConsumer {
|
|||
|
||||
assert 4+normCount*state.numDocs == normsOut.getFilePointer() : ".nrm file size mismatch: expected=" + (4+normCount*state.numDocs) + " actual=" + normsOut.getFilePointer();
|
||||
}
|
||||
|
||||
success = true;
|
||||
} finally {
|
||||
normsOut.close();
|
||||
IOUtils.closeSafely(!success, normsOut);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -30,6 +30,7 @@ import org.apache.lucene.index.codecs.FieldsConsumer;
|
|||
import org.apache.lucene.index.codecs.FieldsProducer;
|
||||
import org.apache.lucene.index.codecs.TermsConsumer;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
/**
|
||||
* Enables native per field codec support. This class selects the codec used to
|
||||
|
@ -61,7 +62,15 @@ final class PerFieldCodecWrapper extends Codec {
|
|||
assert segmentCodecs == state.segmentCodecs;
|
||||
final Codec[] codecs = segmentCodecs.codecs;
|
||||
for (int i = 0; i < codecs.length; i++) {
|
||||
consumers.add(codecs[i].fieldsConsumer(new SegmentWriteState(state, "" + i)));
|
||||
boolean success = false;
|
||||
try {
|
||||
consumers.add(codecs[i].fieldsConsumer(new SegmentWriteState(state, "" + i)));
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeSafely(true, consumers);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -74,22 +83,7 @@ final class PerFieldCodecWrapper extends Codec {
|
|||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
Iterator<FieldsConsumer> it = consumers.iterator();
|
||||
IOException err = null;
|
||||
while (it.hasNext()) {
|
||||
try {
|
||||
it.next().close();
|
||||
} catch (IOException ioe) {
|
||||
// keep first IOException we hit but keep
|
||||
// closing the rest
|
||||
if (err == null) {
|
||||
err = ioe;
|
||||
}
|
||||
}
|
||||
}
|
||||
if (err != null) {
|
||||
throw err;
|
||||
}
|
||||
IOUtils.closeSafely(false, consumers);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -122,14 +116,7 @@ final class PerFieldCodecWrapper extends Codec {
|
|||
// If we hit exception (eg, IOE because writer was
|
||||
// committing, or, for any other reason) we must
|
||||
// go back and close all FieldsProducers we opened:
|
||||
for(FieldsProducer fp : producers.values()) {
|
||||
try {
|
||||
fp.close();
|
||||
} catch (Throwable t) {
|
||||
// Suppress all exceptions here so we continue
|
||||
// to throw the original one
|
||||
}
|
||||
}
|
||||
IOUtils.closeSafely(true, producers.values());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -177,22 +164,7 @@ final class PerFieldCodecWrapper extends Codec {
|
|||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
Iterator<FieldsProducer> it = codecs.values().iterator();
|
||||
IOException err = null;
|
||||
while (it.hasNext()) {
|
||||
try {
|
||||
it.next().close();
|
||||
} catch (IOException ioe) {
|
||||
// keep first IOException we hit but keep
|
||||
// closing the rest
|
||||
if (err == null) {
|
||||
err = ioe;
|
||||
}
|
||||
}
|
||||
}
|
||||
if (err != null) {
|
||||
throw err;
|
||||
}
|
||||
IOUtils.closeSafely(false, codecs.values());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -40,6 +40,7 @@ import org.apache.lucene.store.Directory;
|
|||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.store.NoSuchDirectoryException;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.ThreadInterruptedException;
|
||||
|
||||
/**
|
||||
|
@ -323,17 +324,13 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfo> {
|
|||
SegmentInfosWriter infosWriter = codecs.getSegmentInfosWriter();
|
||||
segnOutput = infosWriter.writeInfos(directory, segmentFileName, this);
|
||||
infosWriter.prepareCommit(segnOutput);
|
||||
success = true;
|
||||
pendingSegnOutput = segnOutput;
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
// We hit an exception above; try to close the file
|
||||
// but suppress any exception:
|
||||
try {
|
||||
segnOutput.close();
|
||||
} catch (Throwable t) {
|
||||
// Suppress so we keep throwing the original exception
|
||||
}
|
||||
IOUtils.closeSafely(true, segnOutput);
|
||||
try {
|
||||
// Try not to leave a truncated segments_N file in
|
||||
// the index:
|
||||
|
@ -945,9 +942,12 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfo> {
|
|||
} finally {
|
||||
genOutput.close();
|
||||
}
|
||||
} catch (ThreadInterruptedException t) {
|
||||
throw t;
|
||||
} catch (Throwable t) {
|
||||
// It's OK if we fail to write this file since it's
|
||||
// used only as one of the retry fallbacks.
|
||||
// nocommit if this is thread interrupted we should rethrow
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -962,7 +962,6 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfo> {
|
|||
prepareCommit(dir);
|
||||
finishCommit(dir);
|
||||
}
|
||||
|
||||
|
||||
public String toString(Directory directory) {
|
||||
StringBuilder buffer = new StringBuilder();
|
||||
|
|
|
@ -22,7 +22,6 @@ import java.util.ArrayList;
|
|||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.regex.Pattern; // for assert
|
||||
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.index.IndexReader.FieldOption;
|
||||
|
@ -34,6 +33,7 @@ import org.apache.lucene.store.Directory;
|
|||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.MultiBits;
|
||||
import org.apache.lucene.util.ReaderUtil;
|
||||
|
||||
|
@ -546,14 +546,13 @@ final class SegmentMerger {
|
|||
}
|
||||
codec = segmentWriteState.segmentCodecs.codec();
|
||||
final FieldsConsumer consumer = codec.fieldsConsumer(segmentWriteState);
|
||||
|
||||
// NOTE: this is silly, yet, necessary -- we create a
|
||||
// MultiBits as our skip docs only to have it broken
|
||||
// apart when we step through the docs enums in
|
||||
// MultiDocsEnum.
|
||||
mergeState.multiDeletedDocs = new MultiBits(bits, bitsStarts);
|
||||
|
||||
try {
|
||||
// NOTE: this is silly, yet, necessary -- we create a
|
||||
// MultiBits as our skip docs only to have it broken
|
||||
// apart when we step through the docs enums in
|
||||
// MultiDocsEnum.
|
||||
mergeState.multiDeletedDocs = new MultiBits(bits, bitsStarts);
|
||||
|
||||
consumer.merge(mergeState,
|
||||
new MultiFields(fields.toArray(Fields.EMPTY_ARRAY),
|
||||
slices.toArray(ReaderUtil.Slice.EMPTY_ARRAY)));
|
||||
|
@ -579,6 +578,7 @@ final class SegmentMerger {
|
|||
|
||||
private void mergeNorms() throws IOException {
|
||||
IndexOutput output = null;
|
||||
boolean success = false;
|
||||
try {
|
||||
for (FieldInfo fi : fieldInfos) {
|
||||
if (fi.isIndexed && !fi.omitNorms) {
|
||||
|
@ -612,10 +612,9 @@ final class SegmentMerger {
|
|||
}
|
||||
}
|
||||
}
|
||||
success = true;
|
||||
} finally {
|
||||
if (output != null) {
|
||||
output.close();
|
||||
}
|
||||
IOUtils.closeSafely(!success, output);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -54,9 +54,7 @@ final class TermVectorsTermsWriter extends TermsHashConsumer {
|
|||
fill(state.numDocs);
|
||||
assert state.segmentName != null;
|
||||
String idxName = IndexFileNames.segmentFileName(state.segmentName, "", IndexFileNames.VECTORS_INDEX_EXTENSION);
|
||||
tvx.close();
|
||||
tvf.close();
|
||||
tvd.close();
|
||||
IOUtils.closeSafely(false, tvx, tvf, tvd);
|
||||
tvx = tvd = tvf = null;
|
||||
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));
|
||||
|
@ -89,18 +87,25 @@ final class TermVectorsTermsWriter extends TermsHashConsumer {
|
|||
|
||||
private final void initTermVectorsWriter() throws IOException {
|
||||
if (tvx == null) {
|
||||
boolean success = false;
|
||||
try {
|
||||
// If we hit an exception while init'ing the term
|
||||
// vector output files, we must abort this segment
|
||||
// because those files will be in an unknown
|
||||
// state:
|
||||
tvx = docWriter.directory.createOutput(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_INDEX_EXTENSION));
|
||||
tvd = docWriter.directory.createOutput(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION));
|
||||
tvf = docWriter.directory.createOutput(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_FIELDS_EXTENSION));
|
||||
|
||||
// If we hit an exception while init'ing the term
|
||||
// vector output files, we must abort this segment
|
||||
// because those files will be in an unknown
|
||||
// state:
|
||||
tvx = docWriter.directory.createOutput(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_INDEX_EXTENSION));
|
||||
tvd = docWriter.directory.createOutput(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION));
|
||||
tvf = docWriter.directory.createOutput(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_FIELDS_EXTENSION));
|
||||
|
||||
tvx.writeInt(TermVectorsReader.FORMAT_CURRENT);
|
||||
tvd.writeInt(TermVectorsReader.FORMAT_CURRENT);
|
||||
tvf.writeInt(TermVectorsReader.FORMAT_CURRENT);
|
||||
tvx.writeInt(TermVectorsReader.FORMAT_CURRENT);
|
||||
tvd.writeInt(TermVectorsReader.FORMAT_CURRENT);
|
||||
tvf.writeInt(TermVectorsReader.FORMAT_CURRENT);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeSafely(true, tvx, tvd, tvf);
|
||||
}
|
||||
}
|
||||
|
||||
lastDocID = 0;
|
||||
}
|
||||
|
@ -152,21 +157,27 @@ final class TermVectorsTermsWriter extends TermsHashConsumer {
|
|||
public void abort() {
|
||||
hasVectors = false;
|
||||
try {
|
||||
IOUtils.closeSafely(tvx, tvd, tvf);
|
||||
} catch (IOException ignored) {
|
||||
IOUtils.closeSafely(true, tvx, tvd, tvf);
|
||||
} catch (IOException e) {
|
||||
// cannot happen since we suppress exceptions
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
|
||||
try {
|
||||
docWriter.directory.deleteFile(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_INDEX_EXTENSION));
|
||||
} catch (IOException ignored) {
|
||||
}
|
||||
|
||||
try {
|
||||
docWriter.directory.deleteFile(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION));
|
||||
} catch (IOException ignored) {
|
||||
}
|
||||
|
||||
try {
|
||||
docWriter.directory.deleteFile(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_FIELDS_EXTENSION));
|
||||
} catch (IOException ignored) {
|
||||
}
|
||||
|
||||
tvx = tvd = tvf = null;
|
||||
lastDocID = 0;
|
||||
|
||||
|
|
|
@ -31,15 +31,22 @@ final class TermVectorsWriter {
|
|||
private FieldInfos fieldInfos;
|
||||
|
||||
public TermVectorsWriter(Directory directory, String segment,
|
||||
FieldInfos fieldInfos)
|
||||
throws IOException {
|
||||
// Open files for TermVector storage
|
||||
tvx = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_INDEX_EXTENSION));
|
||||
tvx.writeInt(TermVectorsReader.FORMAT_CURRENT);
|
||||
tvd = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION));
|
||||
tvd.writeInt(TermVectorsReader.FORMAT_CURRENT);
|
||||
tvf = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_FIELDS_EXTENSION));
|
||||
tvf.writeInt(TermVectorsReader.FORMAT_CURRENT);
|
||||
FieldInfos fieldInfos) throws IOException {
|
||||
boolean success = false;
|
||||
try {
|
||||
// Open files for TermVector storage
|
||||
tvx = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_INDEX_EXTENSION));
|
||||
tvx.writeInt(TermVectorsReader.FORMAT_CURRENT);
|
||||
tvd = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION));
|
||||
tvd.writeInt(TermVectorsReader.FORMAT_CURRENT);
|
||||
tvf = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_FIELDS_EXTENSION));
|
||||
tvf.writeInt(TermVectorsReader.FORMAT_CURRENT);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeSafely(true, tvx, tvd, tvf);
|
||||
}
|
||||
}
|
||||
|
||||
this.fieldInfos = fieldInfos;
|
||||
}
|
||||
|
@ -51,8 +58,7 @@ final class TermVectorsWriter {
|
|||
* @param vectors
|
||||
* @throws IOException
|
||||
*/
|
||||
public final void addAllDocVectors(TermFreqVector[] vectors)
|
||||
throws IOException {
|
||||
public final void addAllDocVectors(TermFreqVector[] vectors) throws IOException {
|
||||
|
||||
tvx.writeLong(tvd.getFilePointer());
|
||||
tvx.writeLong(tvf.getFilePointer());
|
||||
|
@ -187,6 +193,6 @@ final class TermVectorsWriter {
|
|||
final void close() throws IOException {
|
||||
// make an effort to close all streams we can but remember and re-throw
|
||||
// the first exception encountered in this process
|
||||
IOUtils.closeSafely(tvx, tvd, tvf);
|
||||
IOUtils.closeSafely(false, tvx, tvd, tvf);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -54,7 +54,6 @@ final class TermsHash extends InvertedDocConsumer {
|
|||
|
||||
final boolean trackAllocations;
|
||||
|
||||
|
||||
public TermsHash(final DocumentsWriterPerThread docWriter, final TermsHashConsumer consumer, boolean trackAllocations, final TermsHash nextTermsHash) {
|
||||
this.docState = docWriter.docState;
|
||||
this.docWriter = docWriter;
|
||||
|
@ -108,11 +107,11 @@ final class TermsHash extends InvertedDocConsumer {
|
|||
}
|
||||
|
||||
for (final Map.Entry<FieldInfo,InvertedDocConsumerPerField> entry : fieldsToFlush.entrySet()) {
|
||||
TermsHashPerField perField = (TermsHashPerField) entry.getValue();
|
||||
childFields.put(entry.getKey(), perField.consumer);
|
||||
if (nextTermsHash != null) {
|
||||
nextChildFields.put(entry.getKey(), perField.nextPerField);
|
||||
}
|
||||
TermsHashPerField perField = (TermsHashPerField) entry.getValue();
|
||||
childFields.put(entry.getKey(), perField.consumer);
|
||||
if (nextTermsHash != null) {
|
||||
nextChildFields.put(entry.getKey(), perField.nextPerField);
|
||||
}
|
||||
}
|
||||
|
||||
consumer.flush(childFields, state);
|
||||
|
@ -134,12 +133,9 @@ final class TermsHash extends InvertedDocConsumer {
|
|||
|
||||
@Override
|
||||
void finishDocument() throws IOException {
|
||||
try {
|
||||
consumer.finishDocument(this);
|
||||
} finally {
|
||||
if (nextTermsHash != null) {
|
||||
nextTermsHash.consumer.finishDocument(nextTermsHash);
|
||||
}
|
||||
consumer.finishDocument(this);
|
||||
if (nextTermsHash != null) {
|
||||
nextTermsHash.consumer.finishDocument(nextTermsHash);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -31,6 +31,7 @@ import org.apache.lucene.store.RAMOutputStream;
|
|||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.CodecUtil;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
|
||||
// TODO: currently we encode all terms between two indexed
|
||||
|
@ -66,24 +67,29 @@ public class BlockTermsWriter extends FieldsConsumer {
|
|||
|
||||
//private final String segment;
|
||||
|
||||
public BlockTermsWriter(
|
||||
TermsIndexWriterBase termsIndexWriter,
|
||||
SegmentWriteState state,
|
||||
PostingsWriterBase postingsWriter)
|
||||
throws IOException
|
||||
{
|
||||
public BlockTermsWriter(TermsIndexWriterBase termsIndexWriter,
|
||||
SegmentWriteState state, PostingsWriterBase postingsWriter)
|
||||
throws IOException {
|
||||
final String termsFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, TERMS_EXTENSION);
|
||||
this.termsIndexWriter = termsIndexWriter;
|
||||
out = state.directory.createOutput(termsFileName);
|
||||
fieldInfos = state.fieldInfos;
|
||||
writeHeader(out);
|
||||
currentField = null;
|
||||
this.postingsWriter = postingsWriter;
|
||||
//segment = state.segmentName;
|
||||
|
||||
//System.out.println("BTW.init seg=" + state.segmentName);
|
||||
|
||||
postingsWriter.start(out); // have consumer write its format/header
|
||||
boolean success = false;
|
||||
try {
|
||||
fieldInfos = state.fieldInfos;
|
||||
writeHeader(out);
|
||||
currentField = null;
|
||||
this.postingsWriter = postingsWriter;
|
||||
//segment = state.segmentName;
|
||||
|
||||
//System.out.println("BTW.init seg=" + state.segmentName);
|
||||
|
||||
postingsWriter.start(out); // have consumer write its format/header
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeSafely(true, out);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
protected void writeHeader(IndexOutput out) throws IOException {
|
||||
|
@ -130,20 +136,11 @@ public class BlockTermsWriter extends FieldsConsumer {
|
|||
}
|
||||
writeTrailer(dirStart);
|
||||
} finally {
|
||||
try {
|
||||
out.close();
|
||||
} finally {
|
||||
try {
|
||||
postingsWriter.close();
|
||||
} finally {
|
||||
termsIndexWriter.close();
|
||||
}
|
||||
}
|
||||
IOUtils.closeSafely(false, out, postingsWriter, termsIndexWriter);
|
||||
}
|
||||
}
|
||||
|
||||
protected void writeTrailer(long dirStart) throws IOException {
|
||||
// TODO Auto-generated method stub
|
||||
out.seek(CodecUtil.headerLength(CODEC_NAME));
|
||||
out.writeLong(dirStart);
|
||||
}
|
||||
|
|
|
@ -24,6 +24,7 @@ import org.apache.lucene.index.SegmentInfos;
|
|||
import org.apache.lucene.store.ChecksumIndexOutput;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
/**
|
||||
* Default implementation of {@link SegmentInfosWriter}.
|
||||
|
@ -56,16 +57,24 @@ public class DefaultSegmentInfosWriter extends SegmentInfosWriter {
|
|||
public IndexOutput writeInfos(Directory dir, String segmentFileName, SegmentInfos infos)
|
||||
throws IOException {
|
||||
IndexOutput out = createOutput(dir, segmentFileName);
|
||||
out.writeInt(FORMAT_CURRENT); // write FORMAT
|
||||
out.writeLong(infos.version);
|
||||
out.writeInt(infos.counter); // write counter
|
||||
out.writeLong(infos.getGlobalFieldMapVersion());
|
||||
out.writeInt(infos.size()); // write infos
|
||||
for (SegmentInfo si : infos) {
|
||||
si.write(out);
|
||||
boolean success = false;
|
||||
try {
|
||||
out.writeInt(FORMAT_CURRENT); // write FORMAT
|
||||
out.writeLong(infos.version);
|
||||
out.writeInt(infos.counter); // write counter
|
||||
out.writeLong(infos.getGlobalFieldMapVersion());
|
||||
out.writeInt(infos.size()); // write infos
|
||||
for (SegmentInfo si : infos) {
|
||||
si.write(out);
|
||||
}
|
||||
out.writeStringStringMap(infos.getUserData());
|
||||
success = true;
|
||||
return out;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeSafely(true, out);
|
||||
}
|
||||
}
|
||||
out.writeStringStringMap(infos.getUserData());
|
||||
return out;
|
||||
}
|
||||
|
||||
protected IndexOutput createOutput(Directory dir, String segmentFileName)
|
||||
|
|
|
@ -24,6 +24,7 @@ import org.apache.lucene.index.FieldInfo;
|
|||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.CodecUtil;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.PagedBytes;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
|
@ -108,6 +109,7 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
|
|||
}
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) IOUtils.closeSafely(true, in);
|
||||
if (indexDivisor > 0) {
|
||||
in.close();
|
||||
in = null;
|
||||
|
|
|
@ -25,6 +25,7 @@ import org.apache.lucene.index.SegmentWriteState;
|
|||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.CodecUtil;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
import java.util.List;
|
||||
|
@ -58,9 +59,17 @@ public class FixedGapTermsIndexWriter extends TermsIndexWriterBase {
|
|||
final String indexFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, TERMS_INDEX_EXTENSION);
|
||||
termIndexInterval = state.termIndexInterval;
|
||||
out = state.directory.createOutput(indexFileName);
|
||||
fieldInfos = state.fieldInfos;
|
||||
writeHeader(out);
|
||||
out.writeInt(termIndexInterval);
|
||||
boolean success = false;
|
||||
try {
|
||||
fieldInfos = state.fieldInfos;
|
||||
writeHeader(out);
|
||||
out.writeInt(termIndexInterval);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeSafely(true, out);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
protected void writeHeader(IndexOutput out) throws IOException {
|
||||
|
@ -202,33 +211,37 @@ public class FixedGapTermsIndexWriter extends TermsIndexWriterBase {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
final long dirStart = out.getFilePointer();
|
||||
final int fieldCount = fields.size();
|
||||
|
||||
int nonNullFieldCount = 0;
|
||||
for(int i=0;i<fieldCount;i++) {
|
||||
SimpleFieldWriter field = fields.get(i);
|
||||
if (field.numIndexTerms > 0) {
|
||||
nonNullFieldCount++;
|
||||
boolean success = false;
|
||||
try {
|
||||
final long dirStart = out.getFilePointer();
|
||||
final int fieldCount = fields.size();
|
||||
|
||||
int nonNullFieldCount = 0;
|
||||
for(int i=0;i<fieldCount;i++) {
|
||||
SimpleFieldWriter field = fields.get(i);
|
||||
if (field.numIndexTerms > 0) {
|
||||
nonNullFieldCount++;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
out.writeVInt(nonNullFieldCount);
|
||||
for(int i=0;i<fieldCount;i++) {
|
||||
SimpleFieldWriter field = fields.get(i);
|
||||
if (field.numIndexTerms > 0) {
|
||||
out.writeVInt(field.fieldInfo.number);
|
||||
out.writeVInt(field.numIndexTerms);
|
||||
out.writeVLong(field.termsStart);
|
||||
out.writeVLong(field.indexStart);
|
||||
out.writeVLong(field.packedIndexStart);
|
||||
out.writeVLong(field.packedOffsetsStart);
|
||||
|
||||
out.writeVInt(nonNullFieldCount);
|
||||
for(int i=0;i<fieldCount;i++) {
|
||||
SimpleFieldWriter field = fields.get(i);
|
||||
if (field.numIndexTerms > 0) {
|
||||
out.writeVInt(field.fieldInfo.number);
|
||||
out.writeVInt(field.numIndexTerms);
|
||||
out.writeVLong(field.termsStart);
|
||||
out.writeVLong(field.indexStart);
|
||||
out.writeVLong(field.packedIndexStart);
|
||||
out.writeVLong(field.packedOffsetsStart);
|
||||
}
|
||||
}
|
||||
writeTrailer(dirStart);
|
||||
success = true;
|
||||
} finally {
|
||||
IOUtils.closeSafely(!success, out);
|
||||
}
|
||||
writeTrailer(dirStart);
|
||||
out.close();
|
||||
}
|
||||
|
||||
protected void writeTrailer(long dirStart) throws IOException {
|
||||
|
|
|
@ -19,10 +19,12 @@ package org.apache.lucene.index.codecs;
|
|||
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
|
||||
/** @lucene.experimental */
|
||||
public abstract class TermsIndexWriterBase {
|
||||
public abstract class TermsIndexWriterBase implements Closeable {
|
||||
|
||||
public abstract class FieldWriter {
|
||||
public abstract boolean checkIndexTerm(BytesRef text, TermStats stats) throws IOException;
|
||||
|
@ -31,6 +33,4 @@ public abstract class TermsIndexWriterBase {
|
|||
}
|
||||
|
||||
public abstract FieldWriter addField(FieldInfo fieldInfo, long termsFilePointer) throws IOException;
|
||||
|
||||
public abstract void close() throws IOException;
|
||||
}
|
||||
|
|
|
@ -28,6 +28,7 @@ import org.apache.lucene.index.SegmentWriteState;
|
|||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.CodecUtil;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.automaton.fst.Builder;
|
||||
import org.apache.lucene.util.automaton.fst.FST;
|
||||
import org.apache.lucene.util.automaton.fst.PositiveIntOutputs;
|
||||
|
@ -159,9 +160,17 @@ public class VariableGapTermsIndexWriter extends TermsIndexWriterBase {
|
|||
public VariableGapTermsIndexWriter(SegmentWriteState state, IndexTermSelector policy) throws IOException {
|
||||
final String indexFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, TERMS_INDEX_EXTENSION);
|
||||
out = state.directory.createOutput(indexFileName);
|
||||
fieldInfos = state.fieldInfos;
|
||||
this.policy = policy;
|
||||
writeHeader(out);
|
||||
boolean success = false;
|
||||
try {
|
||||
fieldInfos = state.fieldInfos;
|
||||
this.policy = policy;
|
||||
writeHeader(out);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeSafely(true, out);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
protected void writeHeader(IndexOutput out) throws IOException {
|
||||
|
@ -265,8 +274,8 @@ public class VariableGapTermsIndexWriter extends TermsIndexWriterBase {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
try {
|
||||
final long dirStart = out.getFilePointer();
|
||||
final int fieldCount = fields.size();
|
||||
|
||||
|
@ -287,8 +296,10 @@ public class VariableGapTermsIndexWriter extends TermsIndexWriterBase {
|
|||
}
|
||||
}
|
||||
writeTrailer(dirStart);
|
||||
} finally {
|
||||
out.close();
|
||||
}
|
||||
}
|
||||
|
||||
protected void writeTrailer(long dirStart) throws IOException {
|
||||
out.seek(CodecUtil.headerLength(CODEC_NAME));
|
||||
|
|
|
@ -38,6 +38,7 @@ import org.apache.lucene.index.codecs.TermsIndexReaderBase;
|
|||
import org.apache.lucene.index.codecs.TermsIndexWriterBase;
|
||||
import org.apache.lucene.index.codecs.standard.StandardCodec;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
/** This codec "inlines" the postings for terms that have
|
||||
* low docFreq. It wraps another codec, which is used for
|
||||
|
@ -81,7 +82,7 @@ public class PulsingCodec extends Codec {
|
|||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
pulsingWriter.close();
|
||||
IOUtils.closeSafely(true, pulsingWriter);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -93,11 +94,7 @@ public class PulsingCodec extends Codec {
|
|||
return ret;
|
||||
} finally {
|
||||
if (!success) {
|
||||
try {
|
||||
pulsingWriter.close();
|
||||
} finally {
|
||||
indexWriter.close();
|
||||
}
|
||||
IOUtils.closeSafely(true, pulsingWriter, indexWriter);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -71,8 +71,6 @@ public final class PulsingPostingsWriterImpl extends PostingsWriterBase {
|
|||
* for this term) is <= maxPositions, then the postings are
|
||||
* inlined into terms dict */
|
||||
public PulsingPostingsWriterImpl(int maxPositions, PostingsWriterBase wrappedPostingsWriter) throws IOException {
|
||||
super();
|
||||
|
||||
pending = new Position[maxPositions];
|
||||
for(int i=0;i<maxPositions;i++) {
|
||||
pending[i] = new Position();
|
||||
|
|
|
@ -31,6 +31,7 @@ import org.apache.lucene.store.IndexOutput;
|
|||
import org.apache.lucene.store.RAMOutputStream;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.CodecUtil;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
/** Writes frq to .frq, docs to .doc, pos to .pos, payloads
|
||||
* to .pyl, skip data to .skp
|
||||
|
@ -49,18 +50,18 @@ public final class SepPostingsWriterImpl extends PostingsWriterBase {
|
|||
final static int VERSION_START = 0;
|
||||
final static int VERSION_CURRENT = VERSION_START;
|
||||
|
||||
final IntIndexOutput freqOut;
|
||||
final IntIndexOutput.Index freqIndex;
|
||||
IntIndexOutput freqOut;
|
||||
IntIndexOutput.Index freqIndex;
|
||||
|
||||
final IntIndexOutput posOut;
|
||||
final IntIndexOutput.Index posIndex;
|
||||
IntIndexOutput posOut;
|
||||
IntIndexOutput.Index posIndex;
|
||||
|
||||
final IntIndexOutput docOut;
|
||||
final IntIndexOutput.Index docIndex;
|
||||
IntIndexOutput docOut;
|
||||
IntIndexOutput.Index docIndex;
|
||||
|
||||
final IndexOutput payloadOut;
|
||||
IndexOutput payloadOut;
|
||||
|
||||
final IndexOutput skipOut;
|
||||
IndexOutput skipOut;
|
||||
IndexOutput termsOut;
|
||||
|
||||
final SepSkipListWriter skipListWriter;
|
||||
|
@ -107,44 +108,51 @@ public final class SepPostingsWriterImpl extends PostingsWriterBase {
|
|||
}
|
||||
|
||||
public SepPostingsWriterImpl(SegmentWriteState state, IntStreamFactory factory, int skipInterval) throws IOException {
|
||||
super();
|
||||
this.skipInterval = skipInterval;
|
||||
this.skipMinimum = skipInterval; /* set to the same for now */
|
||||
final String docFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, DOC_EXTENSION);
|
||||
docOut = factory.createOutput(state.directory, docFileName);
|
||||
docIndex = docOut.index();
|
||||
freqOut = null;
|
||||
freqIndex = null;
|
||||
posOut = null;
|
||||
posIndex = null;
|
||||
payloadOut = null;
|
||||
boolean success = false;
|
||||
try {
|
||||
this.skipInterval = skipInterval;
|
||||
this.skipMinimum = skipInterval; /* set to the same for now */
|
||||
final String docFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, DOC_EXTENSION);
|
||||
docOut = factory.createOutput(state.directory, docFileName);
|
||||
docIndex = docOut.index();
|
||||
|
||||
if (state.fieldInfos.hasProx()) {
|
||||
final String frqFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, FREQ_EXTENSION);
|
||||
freqOut = factory.createOutput(state.directory, frqFileName);
|
||||
freqIndex = freqOut.index();
|
||||
|
||||
final String posFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, POS_EXTENSION);
|
||||
posOut = factory.createOutput(state.directory, posFileName);
|
||||
posIndex = posOut.index();
|
||||
|
||||
// TODO: -- only if at least one field stores payloads?
|
||||
final String payloadFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, PAYLOAD_EXTENSION);
|
||||
payloadOut = state.directory.createOutput(payloadFileName);
|
||||
}
|
||||
|
||||
final String skipFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, SKIP_EXTENSION);
|
||||
skipOut = state.directory.createOutput(skipFileName);
|
||||
|
||||
totalNumDocs = state.numDocs;
|
||||
|
||||
skipListWriter = new SepSkipListWriter(skipInterval,
|
||||
maxSkipLevels,
|
||||
state.numDocs,
|
||||
freqOut, docOut,
|
||||
posOut, payloadOut);
|
||||
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeSafely(true, docOut, skipOut, freqOut, posOut, payloadOut);
|
||||
}
|
||||
|
||||
if (state.fieldInfos.hasProx()) {
|
||||
final String frqFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, FREQ_EXTENSION);
|
||||
freqOut = factory.createOutput(state.directory, frqFileName);
|
||||
freqIndex = freqOut.index();
|
||||
|
||||
final String posFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, POS_EXTENSION);
|
||||
posOut = factory.createOutput(state.directory, posFileName);
|
||||
posIndex = posOut.index();
|
||||
|
||||
// TODO: -- only if at least one field stores payloads?
|
||||
final String payloadFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, PAYLOAD_EXTENSION);
|
||||
payloadOut = state.directory.createOutput(payloadFileName);
|
||||
|
||||
} else {
|
||||
freqOut = null;
|
||||
freqIndex = null;
|
||||
posOut = null;
|
||||
posIndex = null;
|
||||
payloadOut = null;
|
||||
}
|
||||
|
||||
final String skipFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, SKIP_EXTENSION);
|
||||
skipOut = state.directory.createOutput(skipFileName);
|
||||
|
||||
totalNumDocs = state.numDocs;
|
||||
|
||||
skipListWriter = new SepSkipListWriter(skipInterval,
|
||||
maxSkipLevels,
|
||||
state.numDocs,
|
||||
freqOut, docOut,
|
||||
posOut, payloadOut);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -306,25 +314,7 @@ public final class SepPostingsWriterImpl extends PostingsWriterBase {
|
|||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
try {
|
||||
docOut.close();
|
||||
} finally {
|
||||
try {
|
||||
skipOut.close();
|
||||
} finally {
|
||||
if (freqOut != null) {
|
||||
try {
|
||||
freqOut.close();
|
||||
} finally {
|
||||
try {
|
||||
posOut.close();
|
||||
} finally {
|
||||
payloadOut.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
IOUtils.closeSafely(false, docOut, skipOut, freqOut, posOut, payloadOut);
|
||||
}
|
||||
|
||||
public static void getExtensions(Set<String> extensions) {
|
||||
|
|
|
@ -143,8 +143,11 @@ class SimpleTextFieldsWriter extends FieldsConsumer {
|
|||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
write(END);
|
||||
newline();
|
||||
out.close();
|
||||
try {
|
||||
write(END);
|
||||
newline();
|
||||
} finally {
|
||||
out.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -33,6 +33,7 @@ import org.apache.lucene.store.IndexOutput;
|
|||
import org.apache.lucene.store.RAMOutputStream;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.CodecUtil;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
/** @lucene.experimental */
|
||||
public final class StandardPostingsWriter extends PostingsWriterBase {
|
||||
|
@ -42,8 +43,8 @@ public final class StandardPostingsWriter extends PostingsWriterBase {
|
|||
final static int VERSION_START = 0;
|
||||
final static int VERSION_CURRENT = VERSION_START;
|
||||
|
||||
final IndexOutput freqOut;
|
||||
final IndexOutput proxOut;
|
||||
IndexOutput freqOut;
|
||||
IndexOutput proxOut;
|
||||
final DefaultSkipListWriter skipListWriter;
|
||||
/** Expert: The fraction of TermDocs entries stored in skip tables,
|
||||
* used to accelerate {@link DocsEnum#advance(int)}. Larger values result in
|
||||
|
@ -85,31 +86,35 @@ public final class StandardPostingsWriter extends PostingsWriterBase {
|
|||
public StandardPostingsWriter(SegmentWriteState state) throws IOException {
|
||||
this(state, DEFAULT_SKIP_INTERVAL);
|
||||
}
|
||||
|
||||
public StandardPostingsWriter(SegmentWriteState state, int skipInterval) throws IOException {
|
||||
super();
|
||||
this.skipInterval = skipInterval;
|
||||
this.skipMinimum = skipInterval; /* set to the same for now */
|
||||
//this.segment = state.segmentName;
|
||||
String fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, StandardCodec.FREQ_EXTENSION);
|
||||
freqOut = state.directory.createOutput(fileName);
|
||||
|
||||
if (state.fieldInfos.hasProx()) {
|
||||
// At least one field does not omit TF, so create the
|
||||
// prox file
|
||||
fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, StandardCodec.PROX_EXTENSION);
|
||||
proxOut = state.directory.createOutput(fileName);
|
||||
} else {
|
||||
// Every field omits TF so we will write no prox file
|
||||
proxOut = null;
|
||||
boolean success = false;
|
||||
try {
|
||||
if (state.fieldInfos.hasProx()) {
|
||||
// At least one field does not omit TF, so create the
|
||||
// prox file
|
||||
fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, StandardCodec.PROX_EXTENSION);
|
||||
proxOut = state.directory.createOutput(fileName);
|
||||
} else {
|
||||
// Every field omits TF so we will write no prox file
|
||||
proxOut = null;
|
||||
}
|
||||
|
||||
totalNumDocs = state.numDocs;
|
||||
|
||||
skipListWriter = new DefaultSkipListWriter(skipInterval, maxSkipLevels,
|
||||
state.numDocs, freqOut, proxOut);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeSafely(true, freqOut, proxOut);
|
||||
}
|
||||
}
|
||||
|
||||
totalNumDocs = state.numDocs;
|
||||
|
||||
skipListWriter = new DefaultSkipListWriter(skipInterval,
|
||||
maxSkipLevels,
|
||||
state.numDocs,
|
||||
freqOut,
|
||||
proxOut);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -267,12 +272,6 @@ public final class StandardPostingsWriter extends PostingsWriterBase {
|
|||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
try {
|
||||
freqOut.close();
|
||||
} finally {
|
||||
if (proxOut != null) {
|
||||
proxOut.close();
|
||||
}
|
||||
}
|
||||
IOUtils.closeSafely(false, freqOut, proxOut);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -47,44 +47,113 @@ public final class IOUtils {
|
|||
* @param objects objects to call <tt>close()</tt> on
|
||||
*/
|
||||
public static <E extends Exception> void closeSafely(E priorException, Closeable... objects) throws E, IOException {
|
||||
IOException firstIOE = null;
|
||||
Throwable th = null;
|
||||
|
||||
for (Closeable object : objects) {
|
||||
try {
|
||||
if (object != null)
|
||||
if (object != null) {
|
||||
object.close();
|
||||
} catch (IOException ioe) {
|
||||
if (firstIOE == null)
|
||||
firstIOE = ioe;
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
if (th == null) {
|
||||
th = t;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (priorException != null)
|
||||
if (priorException != null) {
|
||||
throw priorException;
|
||||
else if (firstIOE != null)
|
||||
throw firstIOE;
|
||||
} else if (th != null) {
|
||||
if (th instanceof IOException) throw (IOException) th;
|
||||
if (th instanceof RuntimeException) throw (RuntimeException) th;
|
||||
if (th instanceof Error) throw (Error) th;
|
||||
throw new RuntimeException(th);
|
||||
}
|
||||
}
|
||||
|
||||
/** @see #closeSafely(Exception, Closeable...) */
|
||||
public static <E extends Exception> void closeSafely(E priorException, Iterable<Closeable> objects) throws E, IOException {
|
||||
Throwable th = null;
|
||||
|
||||
for (Closeable object : objects) {
|
||||
try {
|
||||
if (object != null) {
|
||||
object.close();
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
if (th == null) {
|
||||
th = t;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (priorException != null) {
|
||||
throw priorException;
|
||||
} else if (th != null) {
|
||||
if (th instanceof IOException) throw (IOException) th;
|
||||
if (th instanceof RuntimeException) throw (RuntimeException) th;
|
||||
if (th instanceof Error) throw (Error) th;
|
||||
throw new RuntimeException(th);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* <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
|
||||
* 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, and if {@code suppressExceptions} is {@code false},
|
||||
* method either throws the first of suppressed exceptions, or completes
|
||||
* normally.
|
||||
*
|
||||
* @param suppressExceptions
|
||||
* if true then exceptions that occur during close() are suppressed
|
||||
* @param objects
|
||||
* objects to call <tt>close()</tt> on
|
||||
*/
|
||||
public static void closeSafely(Closeable... objects) throws IOException {
|
||||
IOException firstIOE = null;
|
||||
public static void closeSafely(boolean suppressExceptions, Closeable... objects) throws IOException {
|
||||
Throwable th = null;
|
||||
|
||||
for (Closeable object : objects) {
|
||||
try {
|
||||
if (object != null)
|
||||
if (object != null) {
|
||||
object.close();
|
||||
} catch (IOException ioe) {
|
||||
if (firstIOE == null)
|
||||
firstIOE = ioe;
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
if (th == null)
|
||||
th = t;
|
||||
}
|
||||
}
|
||||
|
||||
if (firstIOE != null)
|
||||
throw firstIOE;
|
||||
if (th != null && !suppressExceptions) {
|
||||
if (th instanceof IOException) throw (IOException) th;
|
||||
if (th instanceof RuntimeException) throw (RuntimeException) th;
|
||||
if (th instanceof Error) throw (Error) th;
|
||||
throw new RuntimeException(th);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @see #closeSafely(boolean, Closeable...)
|
||||
*/
|
||||
public static void closeSafely(boolean suppressExceptions, Iterable<? extends Closeable> objects) throws IOException {
|
||||
Throwable th = null;
|
||||
|
||||
for (Closeable object : objects) {
|
||||
try {
|
||||
if (object != null) {
|
||||
object.close();
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
if (th == null)
|
||||
th = t;
|
||||
}
|
||||
}
|
||||
|
||||
if (th != null && !suppressExceptions) {
|
||||
if (th instanceof IOException) throw (IOException) th;
|
||||
if (th instanceof RuntimeException) throw (RuntimeException) th;
|
||||
if (th instanceof Error) throw (Error) th;
|
||||
throw new RuntimeException(th);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -44,6 +44,7 @@ import org.apache.lucene.index.codecs.TermsIndexWriterBase;
|
|||
import org.apache.lucene.index.codecs.standard.StandardCodec;
|
||||
import org.apache.lucene.store.*;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
/**
|
||||
* A silly test codec to verify core support for fixed
|
||||
|
@ -97,15 +98,25 @@ public class MockFixedIntBlockCodec extends Codec {
|
|||
|
||||
@Override
|
||||
public IntIndexOutput createOutput(Directory dir, String fileName) throws IOException {
|
||||
return new FixedIntBlockIndexOutput(dir.createOutput(fileName), blockSize) {
|
||||
@Override
|
||||
protected void flushBlock() throws IOException {
|
||||
for(int i=0;i<buffer.length;i++) {
|
||||
assert buffer[i] >= 0;
|
||||
out.writeVInt(buffer[i]);
|
||||
IndexOutput out = dir.createOutput(fileName);
|
||||
boolean success = false;
|
||||
try {
|
||||
FixedIntBlockIndexOutput ret = new FixedIntBlockIndexOutput(out, blockSize) {
|
||||
@Override
|
||||
protected void flushBlock() throws IOException {
|
||||
for(int i=0;i<buffer.length;i++) {
|
||||
assert buffer[i] >= 0;
|
||||
out.writeVInt(buffer[i]);
|
||||
}
|
||||
}
|
||||
};
|
||||
success = true;
|
||||
return ret;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeSafely(true, out);
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -46,6 +46,7 @@ import org.apache.lucene.store.Directory;
|
|||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
/**
|
||||
* A silly test codec to verify core support for variable
|
||||
|
@ -102,34 +103,42 @@ public class MockVariableIntBlockCodec extends Codec {
|
|||
@Override
|
||||
public IntIndexOutput createOutput(Directory dir, String fileName) throws IOException {
|
||||
final IndexOutput out = dir.createOutput(fileName);
|
||||
out.writeInt(baseBlockSize);
|
||||
return new VariableIntBlockIndexOutput(out, 2*baseBlockSize) {
|
||||
|
||||
int pendingCount;
|
||||
final int[] buffer = new int[2+2*baseBlockSize];
|
||||
|
||||
@Override
|
||||
protected int add(int value) throws IOException {
|
||||
assert value >= 0;
|
||||
buffer[pendingCount++] = value;
|
||||
// silly variable block length int encoder: if
|
||||
// first value <= 3, we write N vints at once;
|
||||
// else, 2*N
|
||||
final int flushAt = buffer[0] <= 3 ? baseBlockSize : 2*baseBlockSize;
|
||||
|
||||
// intentionally be non-causal here:
|
||||
if (pendingCount == flushAt+1) {
|
||||
for(int i=0;i<flushAt;i++) {
|
||||
out.writeVInt(buffer[i]);
|
||||
boolean success = false;
|
||||
try {
|
||||
out.writeInt(baseBlockSize);
|
||||
VariableIntBlockIndexOutput ret = new VariableIntBlockIndexOutput(out, 2*baseBlockSize) {
|
||||
int pendingCount;
|
||||
final int[] buffer = new int[2+2*baseBlockSize];
|
||||
|
||||
@Override
|
||||
protected int add(int value) throws IOException {
|
||||
assert value >= 0;
|
||||
buffer[pendingCount++] = value;
|
||||
// silly variable block length int encoder: if
|
||||
// first value <= 3, we write N vints at once;
|
||||
// else, 2*N
|
||||
final int flushAt = buffer[0] <= 3 ? baseBlockSize : 2*baseBlockSize;
|
||||
|
||||
// intentionally be non-causal here:
|
||||
if (pendingCount == flushAt+1) {
|
||||
for(int i=0;i<flushAt;i++) {
|
||||
out.writeVInt(buffer[i]);
|
||||
}
|
||||
buffer[0] = buffer[flushAt];
|
||||
pendingCount = 1;
|
||||
return flushAt;
|
||||
} else {
|
||||
return 0;
|
||||
}
|
||||
buffer[0] = buffer[flushAt];
|
||||
pendingCount = 1;
|
||||
return flushAt;
|
||||
} else {
|
||||
return 0;
|
||||
}
|
||||
};
|
||||
success = true;
|
||||
return ret;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeSafely(true, out);
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -136,8 +136,11 @@ public class MockRandomCodec extends Codec {
|
|||
|
||||
final String seedFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, SEED_EXT);
|
||||
final IndexOutput out = state.directory.createOutput(seedFileName);
|
||||
out.writeLong(seed);
|
||||
out.close();
|
||||
try {
|
||||
out.writeLong(seed);
|
||||
} finally {
|
||||
out.close();
|
||||
}
|
||||
|
||||
final Random random = new Random(seed);
|
||||
|
||||
|
|
|
@ -20,6 +20,7 @@ package org.apache.lucene.index.codecs.mocksep;
|
|||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.CodecUtil;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.index.codecs.sep.IntIndexOutput;
|
||||
import java.io.IOException;
|
||||
|
||||
|
@ -36,7 +37,15 @@ public class MockSingleIntIndexOutput extends IntIndexOutput {
|
|||
|
||||
public MockSingleIntIndexOutput(Directory dir, String fileName) throws IOException {
|
||||
out = dir.createOutput(fileName);
|
||||
CodecUtil.writeHeader(out, CODEC, VERSION_CURRENT);
|
||||
boolean success = false;
|
||||
try {
|
||||
CodecUtil.writeHeader(out, CODEC, VERSION_CURRENT);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeSafely(true, out);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/** Write an int to the primary file */
|
||||
|
|
|
@ -17,23 +17,24 @@ package org.apache.lucene.index.codecs.preflexrw;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.index.codecs.FieldsConsumer;
|
||||
import org.apache.lucene.index.codecs.TermsConsumer;
|
||||
import org.apache.lucene.index.codecs.PostingsConsumer;
|
||||
import org.apache.lucene.index.codecs.TermStats;
|
||||
import org.apache.lucene.index.codecs.standard.DefaultSkipListWriter;
|
||||
import org.apache.lucene.index.codecs.preflex.PreFlexCodec;
|
||||
import org.apache.lucene.index.CorruptIndexException;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.codecs.preflex.TermInfo;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Comparator;
|
||||
|
||||
import org.apache.lucene.index.CorruptIndexException;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.index.codecs.FieldsConsumer;
|
||||
import org.apache.lucene.index.codecs.PostingsConsumer;
|
||||
import org.apache.lucene.index.codecs.TermStats;
|
||||
import org.apache.lucene.index.codecs.TermsConsumer;
|
||||
import org.apache.lucene.index.codecs.preflex.PreFlexCodec;
|
||||
import org.apache.lucene.index.codecs.preflex.TermInfo;
|
||||
import org.apache.lucene.index.codecs.standard.DefaultSkipListWriter;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
class PreFlexFieldsWriter extends FieldsConsumer {
|
||||
|
||||
private final TermInfosWriter termsOut;
|
||||
|
@ -76,11 +77,7 @@ class PreFlexFieldsWriter extends FieldsConsumer {
|
|||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
termsOut.close();
|
||||
freqOut.close();
|
||||
if (proxOut != null) {
|
||||
proxOut.close();
|
||||
}
|
||||
IOUtils.closeSafely(false, termsOut, freqOut, proxOut);
|
||||
}
|
||||
|
||||
private class PreFlexTermsWriter extends TermsConsumer {
|
||||
|
|
|
@ -18,20 +18,23 @@ package org.apache.lucene.index.codecs.preflexrw;
|
|||
*/
|
||||
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.CharsRef;
|
||||
import org.apache.lucene.util.UnicodeUtil;
|
||||
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.index.codecs.preflex.TermInfo;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.CharsRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.UnicodeUtil;
|
||||
|
||||
|
||||
/** This stores a monotonically increasing set of <Term, TermInfo> pairs in a
|
||||
Directory. A TermInfos can be written once, in order. */
|
||||
|
||||
final class TermInfosWriter {
|
||||
final class TermInfosWriter implements Closeable {
|
||||
/** The file format version, a negative number. */
|
||||
public static final int FORMAT = -3;
|
||||
|
||||
|
@ -84,8 +87,26 @@ final class TermInfosWriter {
|
|||
int interval)
|
||||
throws IOException {
|
||||
initialize(directory, segment, fis, interval, false);
|
||||
boolean success = false;
|
||||
try {
|
||||
other = new TermInfosWriter(directory, segment, fis, interval, true);
|
||||
other.other = this;
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
try {
|
||||
IOUtils.closeSafely(true, output);
|
||||
} catch (IOException e) {
|
||||
// cannot happen since we suppress exceptions
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
|
||||
try {
|
||||
directory.deleteFile(segment + (isIndex ? ".tii" : ".tis"));
|
||||
} catch (IOException ignored) {
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private TermInfosWriter(Directory directory, String segment, FieldInfos fis,
|
||||
|
@ -99,12 +120,30 @@ final class TermInfosWriter {
|
|||
fieldInfos = fis;
|
||||
isIndex = isi;
|
||||
output = directory.createOutput(segment + (isIndex ? ".tii" : ".tis"));
|
||||
boolean success = false;
|
||||
try {
|
||||
output.writeInt(FORMAT_CURRENT); // write format
|
||||
output.writeLong(0); // leave space for size
|
||||
output.writeInt(indexInterval); // write indexInterval
|
||||
output.writeInt(skipInterval); // write skipInterval
|
||||
output.writeInt(maxSkipLevels); // write maxSkipLevels
|
||||
assert initUTF16Results();
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
try {
|
||||
IOUtils.closeSafely(true, output);
|
||||
} catch (IOException e) {
|
||||
// cannot happen since we suppress exceptions
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
|
||||
try {
|
||||
directory.deleteFile(segment + (isIndex ? ".tii" : ".tis"));
|
||||
} catch (IOException ignored) {
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Currently used only by assert statements
|
||||
|
@ -216,13 +255,18 @@ final class TermInfosWriter {
|
|||
}
|
||||
|
||||
/** Called to complete TermInfos creation. */
|
||||
void close() throws IOException {
|
||||
output.seek(4); // write size after format
|
||||
output.writeLong(size);
|
||||
output.close();
|
||||
|
||||
if (!isIndex)
|
||||
other.close();
|
||||
public void close() throws IOException {
|
||||
try {
|
||||
output.seek(4); // write size after format
|
||||
output.writeLong(size);
|
||||
} finally {
|
||||
try {
|
||||
output.close();
|
||||
} finally {
|
||||
if (!isIndex) {
|
||||
other.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -68,24 +68,25 @@ public class MockDirectoryWrapper extends Directory {
|
|||
boolean trackDiskUsage = false;
|
||||
private Set<String> unSyncedFiles;
|
||||
private Set<String> createdFiles;
|
||||
Set<String> openFilesForWrite = new HashSet<String>();
|
||||
private Set<String> openFilesForWrite = new HashSet<String>();
|
||||
Set<String> openLocks = Collections.synchronizedSet(new HashSet<String>());
|
||||
volatile boolean crashed;
|
||||
private ThrottledIndexOutput throttledOutput;
|
||||
private Throttling throttling = Throttling.SOMETIMES;
|
||||
|
||||
// use this for tracking files for crash.
|
||||
// additionally: provides debugging information in case you leave one open
|
||||
Map<Closeable,Exception> openFileHandles = Collections.synchronizedMap(new IdentityHashMap<Closeable,Exception>());
|
||||
private Map<Closeable,Exception> openFileHandles = Collections.synchronizedMap(new IdentityHashMap<Closeable,Exception>());
|
||||
|
||||
// NOTE: we cannot initialize the Map here due to the
|
||||
// order in which our constructor actually does this
|
||||
// member initialization vs when it calls super. It seems
|
||||
// like super is called, then our members are initialized:
|
||||
Map<String,Integer> openFiles;
|
||||
private Map<String,Integer> openFiles;
|
||||
|
||||
// Only tracked if noDeleteOpenFile is true: if an attempt
|
||||
// is made to delete an open file, we enroll it here.
|
||||
Set<String> openFilesDeleted;
|
||||
private Set<String> openFilesDeleted;
|
||||
|
||||
private synchronized void init() {
|
||||
if (openFiles == null) {
|
||||
|
@ -107,6 +108,12 @@ public class MockDirectoryWrapper extends Directory {
|
|||
this.randomState = new Random(random.nextInt());
|
||||
this.throttledOutput = new ThrottledIndexOutput(ThrottledIndexOutput
|
||||
.mBitsToBytes(40 + randomState.nextInt(10)), 5 + randomState.nextInt(5), null);
|
||||
// force wrapping of lockfactory
|
||||
try {
|
||||
setLockFactory(new MockLockFactoryWrapper(this, delegate.getLockFactory()));
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
init();
|
||||
}
|
||||
|
||||
|
@ -127,7 +134,7 @@ public class MockDirectoryWrapper extends Directory {
|
|||
SOMETIMES,
|
||||
/** never throttle output */
|
||||
NEVER
|
||||
};
|
||||
}
|
||||
|
||||
public void setThrottling(Throttling throttling) {
|
||||
this.throttling = throttling;
|
||||
|
@ -208,6 +215,7 @@ public class MockDirectoryWrapper extends Directory {
|
|||
|
||||
public synchronized void clearCrash() throws IOException {
|
||||
crashed = false;
|
||||
openLocks.clear();
|
||||
}
|
||||
|
||||
public void setMaxSizeInBytes(long maxSize) {
|
||||
|
@ -362,9 +370,10 @@ public class MockDirectoryWrapper extends Directory {
|
|||
ramdir.fileMap.put(name, file);
|
||||
}
|
||||
}
|
||||
|
||||
//System.out.println(Thread.currentThread().getName() + ": MDW: create " + name);
|
||||
IndexOutput io = new MockIndexOutputWrapper(this, delegate.createOutput(name), name);
|
||||
openFileHandles.put(io, new RuntimeException("unclosed IndexOutput"));
|
||||
addFileHandle(io, name, false);
|
||||
openFilesForWrite.add(name);
|
||||
|
||||
// throttling REALLY slows down tests, so don't do it very often for SOMETIMES.
|
||||
|
@ -379,6 +388,18 @@ public class MockDirectoryWrapper extends Directory {
|
|||
}
|
||||
}
|
||||
|
||||
private void addFileHandle(Closeable c, String name, boolean input) {
|
||||
Integer v = openFiles.get(name);
|
||||
if (v != null) {
|
||||
v = Integer.valueOf(v.intValue()+1);
|
||||
openFiles.put(name, v);
|
||||
} else {
|
||||
openFiles.put(name, Integer.valueOf(1));
|
||||
}
|
||||
|
||||
openFileHandles.put(c, new RuntimeException("unclosed Index" + (input ? "Input" : "Output") + ": " + name));
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized IndexInput openInput(String name) throws IOException {
|
||||
maybeYield();
|
||||
|
@ -391,16 +412,8 @@ public class MockDirectoryWrapper extends Directory {
|
|||
throw fillOpenTrace(new IOException("MockDirectoryWrapper: file \"" + name + "\" is still open for writing"), name, false);
|
||||
}
|
||||
|
||||
if (openFiles.containsKey(name)) {
|
||||
Integer v = openFiles.get(name);
|
||||
v = Integer.valueOf(v.intValue()+1);
|
||||
openFiles.put(name, v);
|
||||
} else {
|
||||
openFiles.put(name, Integer.valueOf(1));
|
||||
}
|
||||
|
||||
IndexInput ii = new MockIndexInputWrapper(this, name, delegate.openInput(name));
|
||||
openFileHandles.put(ii, new RuntimeException("unclosed IndexInput"));
|
||||
addFileHandle(ii, name, true);
|
||||
return ii;
|
||||
}
|
||||
|
||||
|
@ -447,6 +460,9 @@ public class MockDirectoryWrapper extends Directory {
|
|||
// super() does not throw IOException currently:
|
||||
throw new RuntimeException("MockDirectoryWrapper: cannot close: there are still open files: " + openFiles, cause);
|
||||
}
|
||||
if (noDeleteOpenFile && openLocks.size() > 0) {
|
||||
throw new RuntimeException("MockDirectoryWrapper: cannot close: there are still open locks: " + openLocks);
|
||||
}
|
||||
open = false;
|
||||
if (checkIndexOnClose) {
|
||||
if (LuceneTestCase.VERBOSE) {
|
||||
|
@ -465,6 +481,31 @@ public class MockDirectoryWrapper extends Directory {
|
|||
delegate.close();
|
||||
}
|
||||
|
||||
private synchronized void removeOpenFile(Closeable c, String name) {
|
||||
Integer v = openFiles.get(name);
|
||||
// Could be null when crash() was called
|
||||
if (v != null) {
|
||||
if (v.intValue() == 1) {
|
||||
openFiles.remove(name);
|
||||
openFilesDeleted.remove(name);
|
||||
} else {
|
||||
v = Integer.valueOf(v.intValue()-1);
|
||||
openFiles.put(name, v);
|
||||
}
|
||||
}
|
||||
|
||||
openFileHandles.remove(c);
|
||||
}
|
||||
|
||||
public synchronized void removeIndexOutput(IndexOutput out, String name) {
|
||||
openFilesForWrite.remove(name);
|
||||
removeOpenFile(out, name);
|
||||
}
|
||||
|
||||
public synchronized void removeIndexInput(IndexInput in, String name) {
|
||||
removeOpenFile(in, name);
|
||||
}
|
||||
|
||||
private CodecProvider codecProvider;
|
||||
|
||||
// We pass this CodecProvider to checkIndex when dir is closed...
|
||||
|
|
|
@ -31,8 +31,7 @@ public class MockIndexInputWrapper extends IndexInput {
|
|||
private IndexInput delegate;
|
||||
private boolean isClone;
|
||||
|
||||
/** Construct an empty output buffer.
|
||||
* @throws IOException */
|
||||
/** Construct an empty output buffer. */
|
||||
public MockIndexInputWrapper(MockDirectoryWrapper dir, String name, IndexInput delegate) {
|
||||
this.name = name;
|
||||
this.dir = dir;
|
||||
|
@ -41,24 +40,17 @@ public class MockIndexInputWrapper extends IndexInput {
|
|||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
delegate.close();
|
||||
// Pending resolution on LUCENE-686 we may want to
|
||||
// remove the conditional check so we also track that
|
||||
// all clones get closed:
|
||||
if (!isClone) {
|
||||
synchronized(dir) {
|
||||
Integer v = dir.openFiles.get(name);
|
||||
// Could be null when MockRAMDirectory.crash() was called
|
||||
if (v != null) {
|
||||
if (v.intValue() == 1) {
|
||||
dir.openFiles.remove(name);
|
||||
dir.openFilesDeleted.remove(name);
|
||||
} else {
|
||||
v = Integer.valueOf(v.intValue()-1);
|
||||
dir.openFiles.put(name, v);
|
||||
}
|
||||
}
|
||||
dir.openFileHandles.remove(this);
|
||||
try {
|
||||
// turn on the following to look for leaks closing inputs,
|
||||
// after fixing TestTransactions
|
||||
// dir.maybeThrowDeterministicException();
|
||||
} finally {
|
||||
delegate.close();
|
||||
// Pending resolution on LUCENE-686 we may want to
|
||||
// remove the conditional check so we also track that
|
||||
// all clones get closed:
|
||||
if (!isClone) {
|
||||
dir.removeIndexInput(this, name);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -57,10 +57,7 @@ public class MockIndexOutputWrapper extends IndexOutput {
|
|||
dir.maxUsedSize = size;
|
||||
}
|
||||
}
|
||||
synchronized(dir) {
|
||||
dir.openFileHandles.remove(this);
|
||||
dir.openFilesForWrite.remove(name);
|
||||
}
|
||||
dir.removeIndexOutput(this, name);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,87 @@
|
|||
package org.apache.lucene.store;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public class MockLockFactoryWrapper extends LockFactory {
|
||||
MockDirectoryWrapper dir;
|
||||
LockFactory delegate;
|
||||
|
||||
public MockLockFactoryWrapper(MockDirectoryWrapper dir, LockFactory delegate) {
|
||||
this.dir = dir;
|
||||
this.delegate = delegate;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setLockPrefix(String lockPrefix) {
|
||||
delegate.setLockPrefix(lockPrefix);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getLockPrefix() {
|
||||
return delegate.getLockPrefix();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Lock makeLock(String lockName) {
|
||||
return new MockLock(delegate.makeLock(lockName), lockName);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void clearLock(String lockName) throws IOException {
|
||||
delegate.clearLock(lockName);
|
||||
dir.openLocks.remove(lockName);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "MockLockFactoryWrapper(" + delegate.toString() + ")";
|
||||
}
|
||||
|
||||
private class MockLock extends Lock {
|
||||
private Lock delegateLock;
|
||||
private String name;
|
||||
|
||||
MockLock(Lock delegate, String name) {
|
||||
this.delegateLock = delegate;
|
||||
this.name = name;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean obtain() throws IOException {
|
||||
if (delegateLock.obtain()) {
|
||||
dir.openLocks.add(name);
|
||||
return true;
|
||||
} else {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void release() throws IOException {
|
||||
delegateLock.release();
|
||||
dir.openLocks.remove(name);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isLocked() throws IOException {
|
||||
return delegateLock.isLocked();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -469,6 +469,7 @@ public abstract class LuceneTestCase extends Assert {
|
|||
if (ste.getClassName().indexOf("org.apache.lucene") == -1) break;
|
||||
System.err.println("\t" + ste);
|
||||
}
|
||||
fail("could not remove temp dir: " + entry.getKey());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -971,7 +972,10 @@ public abstract class LuceneTestCase extends Assert {
|
|||
|
||||
clazz = Class.forName(fsdirClass).asSubclass(FSDirectory.class);
|
||||
}
|
||||
MockDirectoryWrapper dir = new MockDirectoryWrapper(random, newFSDirectoryImpl(clazz, f, lf));
|
||||
MockDirectoryWrapper dir = new MockDirectoryWrapper(random, newFSDirectoryImpl(clazz, f));
|
||||
if (lf != null) {
|
||||
dir.setLockFactory(lf);
|
||||
}
|
||||
stores.put(dir, Thread.currentThread().getStackTrace());
|
||||
return dir;
|
||||
} catch (Exception e) {
|
||||
|
@ -1105,7 +1109,7 @@ public abstract class LuceneTestCase extends Assert {
|
|||
}
|
||||
|
||||
private static Directory newFSDirectoryImpl(
|
||||
Class<? extends FSDirectory> clazz, File file, LockFactory lockFactory)
|
||||
Class<? extends FSDirectory> clazz, File file)
|
||||
throws IOException {
|
||||
FSDirectory d = null;
|
||||
try {
|
||||
|
@ -1116,9 +1120,6 @@ public abstract class LuceneTestCase extends Assert {
|
|||
} catch (Exception e) {
|
||||
d = FSDirectory.open(file);
|
||||
}
|
||||
if (lockFactory != null) {
|
||||
d.setLockFactory(lockFactory);
|
||||
}
|
||||
return d;
|
||||
}
|
||||
|
||||
|
@ -1140,7 +1141,7 @@ public abstract class LuceneTestCase extends Assert {
|
|||
tmpFile.delete();
|
||||
tmpFile.mkdir();
|
||||
registerTempFile(tmpFile);
|
||||
return newFSDirectoryImpl(clazz.asSubclass(FSDirectory.class), tmpFile, null);
|
||||
return newFSDirectoryImpl(clazz.asSubclass(FSDirectory.class), tmpFile);
|
||||
}
|
||||
|
||||
// try empty ctor
|
||||
|
|
|
@ -73,9 +73,11 @@ public class ThrottledIndexOutput extends IndexOutput {
|
|||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
sleep(closeDelayMillis + getDelay(true));
|
||||
delegate.close();
|
||||
|
||||
try {
|
||||
sleep(closeDelayMillis + getDelay(true));
|
||||
} finally {
|
||||
delegate.close();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -176,6 +176,7 @@ public class TestCrash extends LuceneTestCase {
|
|||
reader = IndexReader.open(dir, false);
|
||||
assertEquals(157, reader.numDocs());
|
||||
reader.close();
|
||||
dir.clearCrash();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
|
|
|
@ -1688,6 +1688,7 @@ public class TestIndexReader extends LuceneTestCase
|
|||
r2.close();
|
||||
assertTrue(ints == ints2);
|
||||
|
||||
writer.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
|
@ -1735,6 +1736,7 @@ public class TestIndexReader extends LuceneTestCase
|
|||
assertTrue(((SegmentReader) subs[1]).readOnly);
|
||||
assertTrue(ints == ints2);
|
||||
|
||||
writer.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
|
|
|
@ -2123,6 +2123,10 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
allowInterrupt = true;
|
||||
}
|
||||
} catch (ThreadInterruptedException re) {
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: got interrupt");
|
||||
re.printStackTrace(System.out);
|
||||
}
|
||||
Throwable e = re.getCause();
|
||||
assertTrue(e instanceof InterruptedException);
|
||||
if (finish) {
|
||||
|
@ -2720,7 +2724,7 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
// or, at most the write.lock file
|
||||
final int extraFileCount;
|
||||
if (files.length == 1) {
|
||||
assertEquals("write.lock", files[0]);
|
||||
assertTrue(files[0].endsWith("write.lock"));
|
||||
extraFileCount = 1;
|
||||
} else {
|
||||
assertEquals(0, files.length);
|
||||
|
|
|
@ -833,6 +833,10 @@ public class TestIndexWriterDelete extends LuceneTestCase {
|
|||
try {
|
||||
modifier.addDocument(doc);
|
||||
} catch (IOException io) {
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: got expected exc:");
|
||||
io.printStackTrace(System.out);
|
||||
}
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -919,7 +919,7 @@ public class TestIndexWriterExceptions extends LuceneTestCase {
|
|||
assertTrue(failure.failOnCommit && failure.failOnDeleteFile);
|
||||
w.rollback();
|
||||
assertFalse(dir.fileExists("1.fnx"));
|
||||
// FIXME: on windows, this often fails! assertEquals(0, dir.listAll().length);
|
||||
assertEquals(0, dir.listAll().length);
|
||||
dir.close();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -65,6 +65,10 @@ public class TestIndexWriterWithThreads extends LuceneTestCase {
|
|||
writer.updateDocument(new Term("id", ""+(idUpto++)), doc);
|
||||
addCount++;
|
||||
} catch (IOException ioe) {
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: expected exc:");
|
||||
ioe.printStackTrace(System.out);
|
||||
}
|
||||
//System.out.println(Thread.currentThread().getName() + ": hit exc");
|
||||
//ioe.printStackTrace(System.out);
|
||||
if (ioe.getMessage().startsWith("fake disk full at") ||
|
||||
|
@ -218,6 +222,9 @@ public class TestIndexWriterWithThreads extends LuceneTestCase {
|
|||
int NUM_THREADS = 3;
|
||||
|
||||
for(int iter=0;iter<2;iter++) {
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: iter=" + iter);
|
||||
}
|
||||
MockDirectoryWrapper dir = newDirectory();
|
||||
|
||||
IndexWriter writer = new IndexWriter(
|
||||
|
@ -228,6 +235,7 @@ public class TestIndexWriterWithThreads extends LuceneTestCase {
|
|||
setMergePolicy(newLogMergePolicy(4))
|
||||
);
|
||||
((ConcurrentMergeScheduler) writer.getConfig().getMergeScheduler()).setSuppressExceptions();
|
||||
writer.setInfoStream(VERBOSE ? System.out : null);
|
||||
|
||||
IndexerThread[] threads = new IndexerThread[NUM_THREADS];
|
||||
|
||||
|
|
|
@ -217,6 +217,7 @@ public class TestFieldCache extends LuceneTestCase {
|
|||
IndexReader r = IndexReader.open(writer, true);
|
||||
FieldCache.DocTerms terms = FieldCache.DEFAULT.getTerms(r, "foobar");
|
||||
FieldCache.DocTermsIndex termsIndex = FieldCache.DEFAULT.getTermsIndex(r, "foobar");
|
||||
writer.close();
|
||||
r.close();
|
||||
dir.close();
|
||||
}
|
||||
|
|
|
@ -255,15 +255,21 @@ public class TestLockFactory extends LuceneTestCase {
|
|||
// write.lock is stored in index):
|
||||
public void testDefaultFSLockFactoryPrefix() throws IOException {
|
||||
|
||||
// Make sure we get null prefix:
|
||||
// Make sure we get null prefix, which wont happen if setLockFactory is ever called.
|
||||
File dirName = _TestUtil.getTempDir("TestLockFactory.10");
|
||||
Directory dir = newFSDirectory(dirName);
|
||||
|
||||
String prefix = dir.getLockFactory().getLockPrefix();
|
||||
|
||||
assertTrue("Default lock prefix should be null", null == prefix);
|
||||
|
||||
Directory dir = new SimpleFSDirectory(dirName);
|
||||
assertNull("Default lock prefix should be null", dir.getLockFactory().getLockPrefix());
|
||||
dir.close();
|
||||
|
||||
dir = new MMapDirectory(dirName);
|
||||
assertNull("Default lock prefix should be null", dir.getLockFactory().getLockPrefix());
|
||||
dir.close();
|
||||
|
||||
dir = new NIOFSDirectory(dirName);
|
||||
assertNull("Default lock prefix should be null", dir.getLockFactory().getLockPrefix());
|
||||
dir.close();
|
||||
|
||||
_TestUtil.rmDir(dirName);
|
||||
}
|
||||
|
||||
|
|
|
@ -510,7 +510,7 @@ public class FSTLookup extends Lookup {
|
|||
this.automaton = new FST<Object>(new InputStreamDataInput(is), NoOutputs.getSingleton());
|
||||
cacheRootArcs();
|
||||
} finally {
|
||||
IOUtils.closeSafely(is);
|
||||
IOUtils.closeSafely(false, is);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
@ -532,7 +532,7 @@ public class FSTLookup extends Lookup {
|
|||
try {
|
||||
this.automaton.save(new OutputStreamDataOutput(os));
|
||||
} finally {
|
||||
IOUtils.closeSafely(os);
|
||||
IOUtils.closeSafely(false, os);
|
||||
}
|
||||
|
||||
return true;
|
||||
|
|
Loading…
Reference in New Issue