mirror of https://github.com/apache/lucene.git
LUCENE-6124: fix broken close() methods in .store/.codec apis
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1646854 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
1725dabf45
commit
2bafbb726b
|
@ -381,6 +381,9 @@ Bug Fixes
|
|||
* LUCENE-6105: Don't cache FST root arcs if the number of root arcs is
|
||||
small, or if the cache would be > 20% of the size of the FST.
|
||||
(Robert Muir, Mike McCandless)
|
||||
|
||||
* LUCENE-6124: Fix double-close() problems in codec and store APIs.
|
||||
(Robert Muir)
|
||||
|
||||
Documentation
|
||||
|
||||
|
|
|
@ -909,9 +909,15 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
|
|||
private final RAMOutputStream bytesWriter = new RAMOutputStream();
|
||||
}
|
||||
|
||||
private boolean closed;
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
|
||||
if (closed) {
|
||||
return;
|
||||
}
|
||||
closed = true;
|
||||
|
||||
boolean success = false;
|
||||
try {
|
||||
|
||||
|
|
|
@ -483,9 +483,14 @@ public final class BloomFilteringPostingsFormat extends PostingsFormat {
|
|||
}
|
||||
}
|
||||
|
||||
private boolean closed;
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
|
||||
if (closed) {
|
||||
return;
|
||||
}
|
||||
closed = true;
|
||||
delegateFieldsConsumer.close();
|
||||
|
||||
// Now we are done accumulating values for these fields
|
||||
|
@ -499,9 +504,7 @@ public final class BloomFilteringPostingsFormat extends PostingsFormat {
|
|||
}
|
||||
String bloomFileName = IndexFileNames.segmentFileName(
|
||||
state.segmentInfo.name, state.segmentSuffix, BLOOM_EXTENSION);
|
||||
IndexOutput bloomOutput = null;
|
||||
try {
|
||||
bloomOutput = state.directory.createOutput(bloomFileName, state.context);
|
||||
try (IndexOutput bloomOutput = state.directory.createOutput(bloomFileName, state.context)) {
|
||||
CodecUtil.writeIndexHeader(bloomOutput, BLOOM_CODEC_NAME, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
|
||||
// remember the name of the postings format we will delegate to
|
||||
bloomOutput.writeString(delegatePostingsFormat.getName());
|
||||
|
@ -515,8 +518,6 @@ public final class BloomFilteringPostingsFormat extends PostingsFormat {
|
|||
saveAppropriatelySizedBloomFilter(bloomOutput, bloomFilter, fieldInfo);
|
||||
}
|
||||
CodecUtil.writeFooter(bloomOutput);
|
||||
} finally {
|
||||
IOUtils.close(bloomOutput);
|
||||
}
|
||||
//We are done with large bitsets so no need to keep them hanging around
|
||||
bloomFilters.clear();
|
||||
|
|
|
@ -404,14 +404,19 @@ public final class MemoryPostingsFormat extends PostingsFormat {
|
|||
}
|
||||
}
|
||||
|
||||
private boolean closed;
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (closed) {
|
||||
return;
|
||||
}
|
||||
closed = true;
|
||||
|
||||
// EOF marker:
|
||||
try {
|
||||
try (IndexOutput out = this.out) {
|
||||
out.writeVInt(0);
|
||||
CodecUtil.writeFooter(out);
|
||||
} finally {
|
||||
out.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -957,9 +957,15 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
|
|||
private final RAMOutputStream bytesWriter = new RAMOutputStream();
|
||||
}
|
||||
|
||||
private boolean closed;
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
|
||||
if (closed) {
|
||||
return;
|
||||
}
|
||||
closed = true;
|
||||
|
||||
boolean success = false;
|
||||
try {
|
||||
|
||||
|
|
|
@ -200,11 +200,21 @@ public class NRTCachingDirectory extends FilterDirectory implements Accountable
|
|||
// it for defensive reasons... or in case the app is
|
||||
// doing something custom (creating outputs directly w/o
|
||||
// using IndexWriter):
|
||||
for(String fileName : cache.listAll()) {
|
||||
unCache(fileName);
|
||||
boolean success = false;
|
||||
try {
|
||||
if (cache.isOpen) {
|
||||
for(String fileName : cache.listAll()) {
|
||||
unCache(fileName);
|
||||
}
|
||||
}
|
||||
success = true;
|
||||
} finally {
|
||||
if (success) {
|
||||
IOUtils.close(cache, in);
|
||||
} else {
|
||||
IOUtils.closeWhileHandlingException(cache, in);
|
||||
}
|
||||
}
|
||||
cache.close();
|
||||
in.close();
|
||||
}
|
||||
|
||||
/** Subclass can override this to customize logic; return
|
||||
|
|
|
@ -21,6 +21,7 @@ import java.io.IOException;
|
|||
|
||||
import org.apache.lucene.index.IndexWriter;
|
||||
import org.apache.lucene.index.IndexWriterConfig;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
|
||||
public class TestMockDirectoryWrapper extends LuceneTestCase {
|
||||
|
@ -33,9 +34,9 @@ public class TestMockDirectoryWrapper extends LuceneTestCase {
|
|||
fail();
|
||||
} catch (Exception expected) {
|
||||
assertTrue(expected.getMessage().contains("there are still open locks"));
|
||||
} finally {
|
||||
IOUtils.closeWhileHandlingException(iw);
|
||||
}
|
||||
iw.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public void testFailIfIndexWriterNotClosedChangeLockFactory() throws IOException {
|
||||
|
@ -46,9 +47,9 @@ public class TestMockDirectoryWrapper extends LuceneTestCase {
|
|||
fail();
|
||||
} catch (Exception expected) {
|
||||
assertTrue(expected.getMessage().contains("there are still open locks"));
|
||||
} finally {
|
||||
IOUtils.closeWhileHandlingException(iw);
|
||||
}
|
||||
iw.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public void testDiskFull() throws IOException {
|
||||
|
|
|
@ -849,8 +849,14 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
|
|||
private final RAMOutputStream bytesWriter = new RAMOutputStream();
|
||||
}
|
||||
|
||||
private boolean closed;
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (closed) {
|
||||
return;
|
||||
}
|
||||
closed = true;
|
||||
|
||||
boolean success = false;
|
||||
try {
|
||||
|
|
|
@ -208,6 +208,7 @@ public class AssertingDocValuesFormat extends DocValuesFormat {
|
|||
@Override
|
||||
public void close() throws IOException {
|
||||
in.close();
|
||||
in.close(); // close again
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -280,6 +281,7 @@ public class AssertingDocValuesFormat extends DocValuesFormat {
|
|||
@Override
|
||||
public void close() throws IOException {
|
||||
in.close();
|
||||
in.close(); // close again
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -75,6 +75,7 @@ public class AssertingNormsFormat extends NormsFormat {
|
|||
@Override
|
||||
public void close() throws IOException {
|
||||
in.close();
|
||||
in.close(); // close again
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -102,6 +103,7 @@ public class AssertingNormsFormat extends NormsFormat {
|
|||
@Override
|
||||
public void close() throws IOException {
|
||||
in.close();
|
||||
in.close(); // close again
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -72,6 +72,7 @@ public final class AssertingPostingsFormat extends PostingsFormat {
|
|||
@Override
|
||||
public void close() throws IOException {
|
||||
in.close();
|
||||
in.close(); // close again
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -240,6 +241,7 @@ public final class AssertingPostingsFormat extends PostingsFormat {
|
|||
@Override
|
||||
public void close() throws IOException {
|
||||
in.close();
|
||||
in.close(); // close again
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -64,6 +64,7 @@ public class AssertingStoredFieldsFormat extends StoredFieldsFormat {
|
|||
@Override
|
||||
public void close() throws IOException {
|
||||
in.close();
|
||||
in.close(); // close again
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -152,6 +153,7 @@ public class AssertingStoredFieldsFormat extends StoredFieldsFormat {
|
|||
@Override
|
||||
public void close() throws IOException {
|
||||
in.close();
|
||||
in.close(); // close again
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -63,6 +63,7 @@ public class AssertingTermVectorsFormat extends TermVectorsFormat {
|
|||
@Override
|
||||
public void close() throws IOException {
|
||||
in.close();
|
||||
in.close(); // close again
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -206,6 +207,7 @@ public class AssertingTermVectorsFormat extends TermVectorsFormat {
|
|||
@Override
|
||||
public void close() throws IOException {
|
||||
in.close();
|
||||
in.close(); // close again
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -18,7 +18,6 @@ package org.apache.lucene.index;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.lang.reflect.Field;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
|
@ -31,16 +30,37 @@ import java.util.Set;
|
|||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.codecs.DocValuesProducer;
|
||||
import org.apache.lucene.codecs.FieldsConsumer;
|
||||
import org.apache.lucene.codecs.FieldsProducer;
|
||||
import org.apache.lucene.codecs.NormsConsumer;
|
||||
import org.apache.lucene.codecs.NormsProducer;
|
||||
import org.apache.lucene.codecs.StoredFieldsReader;
|
||||
import org.apache.lucene.codecs.StoredFieldsWriter;
|
||||
import org.apache.lucene.codecs.TermVectorsReader;
|
||||
import org.apache.lucene.codecs.TermVectorsWriter;
|
||||
import org.apache.lucene.codecs.mockrandom.MockRandomPostingsFormat;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.FieldType;
|
||||
import org.apache.lucene.document.NumericDocValuesField;
|
||||
import org.apache.lucene.document.TextField;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.FlushInfo;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.store.MockDirectoryWrapper;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.CloseableThreadLocal;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.InfoStream;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.RamUsageTester;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
import org.apache.lucene.util.Version;
|
||||
|
||||
/**
|
||||
* Common tests to all index formats.
|
||||
|
@ -83,7 +103,7 @@ abstract class BaseIndexFileFormatTestCase extends LuceneTestCase {
|
|||
this.root = root;
|
||||
}
|
||||
|
||||
public long accumulateObject(Object o, long shallowSize, Map<Field, Object> fieldValues, Collection<Object> queue) {
|
||||
public long accumulateObject(Object o, long shallowSize, Map<java.lang.reflect.Field, Object> fieldValues, Collection<Object> queue) {
|
||||
for (Class<?> clazz = o.getClass(); clazz != null; clazz = clazz.getSuperclass()) {
|
||||
if (EXCLUDED_CLASSES.contains(clazz) && o != root) {
|
||||
return 0;
|
||||
|
@ -257,5 +277,105 @@ abstract class BaseIndexFileFormatTestCase extends LuceneTestCase {
|
|||
reader2.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
/** Calls close multiple times on closeable codec apis */
|
||||
public void testMultiClose() throws IOException {
|
||||
// first make a one doc index
|
||||
Directory oneDocIndex = newDirectory();
|
||||
IndexWriter iw = new IndexWriter(oneDocIndex, new IndexWriterConfig(new MockAnalyzer(random())));
|
||||
Document oneDoc = new Document();
|
||||
FieldType customType = new FieldType(TextField.TYPE_STORED);
|
||||
customType.setStoreTermVectors(true);
|
||||
Field customField = new Field("field", "contents", customType);
|
||||
oneDoc.add(customField);
|
||||
oneDoc.add(new NumericDocValuesField("field", 5));
|
||||
iw.addDocument(oneDoc);
|
||||
LeafReader oneDocReader = getOnlySegmentReader(DirectoryReader.open(iw, true));
|
||||
iw.close();
|
||||
|
||||
// now feed to codec apis manually
|
||||
// we use FSDir, things like ramdir are not guaranteed to cause fails if you write to them after close(), etc
|
||||
Directory dir = newFSDirectory(createTempDir("justSoYouGetSomeChannelErrors"));
|
||||
Codec codec = getCodec();
|
||||
|
||||
SegmentInfo segmentInfo = new SegmentInfo(dir, Version.LATEST, "_0", 1, false, codec, null, StringHelper.randomId(), new HashMap<>());
|
||||
FieldInfo proto = oneDocReader.getFieldInfos().fieldInfo("field");
|
||||
FieldInfo field = new FieldInfo(proto.name, proto.number, proto.hasVectors(), proto.omitsNorms(), proto.hasPayloads(),
|
||||
proto.getIndexOptions(), proto.getDocValuesType(), proto.getDocValuesGen(), null);
|
||||
|
||||
FieldInfos fieldInfos = new FieldInfos(new FieldInfo[] { field } );
|
||||
|
||||
SegmentWriteState writeState = new SegmentWriteState(null, dir,
|
||||
segmentInfo, fieldInfos,
|
||||
null, new IOContext(new FlushInfo(1, 20)));
|
||||
|
||||
SegmentReadState readState = new SegmentReadState(dir, segmentInfo, fieldInfos, IOContext.READ);
|
||||
|
||||
// PostingsFormat
|
||||
try (FieldsConsumer consumer = codec.postingsFormat().fieldsConsumer(writeState)) {
|
||||
consumer.write(oneDocReader.fields());
|
||||
IOUtils.close(consumer);
|
||||
IOUtils.close(consumer);
|
||||
}
|
||||
try (FieldsProducer producer = codec.postingsFormat().fieldsProducer(readState)) {
|
||||
IOUtils.close(producer);
|
||||
IOUtils.close(producer);
|
||||
}
|
||||
|
||||
// DocValuesFormat
|
||||
try (DocValuesConsumer consumer = codec.docValuesFormat().fieldsConsumer(writeState)) {
|
||||
consumer.addNumericField(field, Collections.singleton(5));
|
||||
IOUtils.close(consumer);
|
||||
IOUtils.close(consumer);
|
||||
}
|
||||
try (DocValuesProducer producer = codec.docValuesFormat().fieldsProducer(readState)) {
|
||||
IOUtils.close(producer);
|
||||
IOUtils.close(producer);
|
||||
}
|
||||
|
||||
// NormsFormat
|
||||
try (NormsConsumer consumer = codec.normsFormat().normsConsumer(writeState)) {
|
||||
consumer.addNormsField(field, Collections.singleton(5));
|
||||
IOUtils.close(consumer);
|
||||
IOUtils.close(consumer);
|
||||
}
|
||||
try (NormsProducer producer = codec.normsFormat().normsProducer(readState)) {
|
||||
IOUtils.close(producer);
|
||||
IOUtils.close(producer);
|
||||
}
|
||||
|
||||
// TermVectorsFormat
|
||||
try (TermVectorsWriter consumer = codec.termVectorsFormat().vectorsWriter(dir, segmentInfo, writeState.context)) {
|
||||
consumer.startDocument(1);
|
||||
consumer.startField(field, 1, false, false, false);
|
||||
consumer.startTerm(new BytesRef("testing"), 2);
|
||||
consumer.finishTerm();
|
||||
consumer.finishField();
|
||||
consumer.finishDocument();
|
||||
consumer.finish(fieldInfos, 1);
|
||||
IOUtils.close(consumer);
|
||||
IOUtils.close(consumer);
|
||||
}
|
||||
try (TermVectorsReader producer = codec.termVectorsFormat().vectorsReader(dir, segmentInfo, fieldInfos, readState.context)) {
|
||||
IOUtils.close(producer);
|
||||
IOUtils.close(producer);
|
||||
}
|
||||
|
||||
// StoredFieldsFormat
|
||||
try (StoredFieldsWriter consumer = codec.storedFieldsFormat().fieldsWriter(dir, segmentInfo, writeState.context)) {
|
||||
consumer.startDocument();
|
||||
consumer.writeField(field, customField);
|
||||
consumer.finishDocument();
|
||||
consumer.finish(fieldInfos, 1);
|
||||
IOUtils.close(consumer);
|
||||
IOUtils.close(consumer);
|
||||
}
|
||||
try (StoredFieldsReader producer = codec.storedFieldsFormat().fieldsReader(dir, segmentInfo, fieldInfos, readState.context)) {
|
||||
IOUtils.close(producer);
|
||||
IOUtils.close(producer);
|
||||
}
|
||||
|
||||
IOUtils.close(oneDocReader, oneDocIndex, dir);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -1040,5 +1040,35 @@ public abstract class BaseDirectoryTestCase extends LuceneTestCase {
|
|||
out.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public void testDoubleCloseDirectory() throws Throwable {
|
||||
Directory dir = getDirectory(createTempDir());
|
||||
IndexOutput out = dir.createOutput("foobar", newIOContext(random()));
|
||||
out.writeString("testing");
|
||||
out.close();
|
||||
dir.close();
|
||||
dir.close(); // close again
|
||||
}
|
||||
|
||||
public void testDoubleCloseOutput() throws Throwable {
|
||||
Directory dir = getDirectory(createTempDir());
|
||||
IndexOutput out = dir.createOutput("foobar", newIOContext(random()));
|
||||
out.writeString("testing");
|
||||
out.close();
|
||||
out.close(); // close again
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public void testDoubleCloseInput() throws Throwable {
|
||||
Directory dir = getDirectory(createTempDir());
|
||||
IndexOutput out = dir.createOutput("foobar", newIOContext(random()));
|
||||
out.writeString("testing");
|
||||
out.close();
|
||||
IndexInput in = dir.openInput("foobar", newIOContext(random()));
|
||||
assertEquals("testing", in.readString());
|
||||
in.close();
|
||||
in.close(); // close again
|
||||
dir.close();
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -730,158 +730,173 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
|
|||
|
||||
@Override
|
||||
public synchronized void close() throws IOException {
|
||||
// files that we tried to delete, but couldn't because readers were open.
|
||||
// all that matters is that we tried! (they will eventually go away)
|
||||
// still open when we tried to delete
|
||||
Set<String> pendingDeletions = new HashSet<>(openFilesDeleted);
|
||||
// virus scanner when we tried to delete
|
||||
pendingDeletions.addAll(triedToDelete);
|
||||
maybeYield();
|
||||
if (openFiles == null) {
|
||||
openFiles = new HashMap<>();
|
||||
openFilesDeleted = new HashSet<>();
|
||||
}
|
||||
if (openFiles.size() > 0) {
|
||||
// print the first one as its very verbose otherwise
|
||||
Exception cause = null;
|
||||
Iterator<Exception> stacktraces = openFileHandles.values().iterator();
|
||||
if (stacktraces.hasNext()) {
|
||||
cause = stacktraces.next();
|
||||
}
|
||||
// RuntimeException instead of IOException because
|
||||
// super() does not throw IOException currently:
|
||||
throw new RuntimeException("MockDirectoryWrapper: cannot close: there are still open files: " + openFiles, cause);
|
||||
}
|
||||
if (openLocks.size() > 0) {
|
||||
Exception cause = null;
|
||||
Iterator<Exception> stacktraces = openLocks.values().iterator();
|
||||
if (stacktraces.hasNext()) {
|
||||
cause = stacktraces.next();
|
||||
}
|
||||
throw new RuntimeException("MockDirectoryWrapper: cannot close: there are still open locks: " + openLocks, cause);
|
||||
if (isOpen) {
|
||||
isOpen = false;
|
||||
} else {
|
||||
in.close(); // but call it again on our wrapped dir
|
||||
return;
|
||||
}
|
||||
|
||||
isOpen = false;
|
||||
if (getCheckIndexOnClose()) {
|
||||
randomIOExceptionRate = 0.0;
|
||||
randomIOExceptionRateOnOpen = 0.0;
|
||||
if (DirectoryReader.indexExists(this)) {
|
||||
if (LuceneTestCase.VERBOSE) {
|
||||
System.out.println("\nNOTE: MockDirectoryWrapper: now crush");
|
||||
boolean success = false;
|
||||
try {
|
||||
// files that we tried to delete, but couldn't because readers were open.
|
||||
// all that matters is that we tried! (they will eventually go away)
|
||||
// still open when we tried to delete
|
||||
Set<String> pendingDeletions = new HashSet<>(openFilesDeleted);
|
||||
// virus scanner when we tried to delete
|
||||
pendingDeletions.addAll(triedToDelete);
|
||||
maybeYield();
|
||||
if (openFiles == null) {
|
||||
openFiles = new HashMap<>();
|
||||
openFilesDeleted = new HashSet<>();
|
||||
}
|
||||
if (openFiles.size() > 0) {
|
||||
// print the first one as its very verbose otherwise
|
||||
Exception cause = null;
|
||||
Iterator<Exception> stacktraces = openFileHandles.values().iterator();
|
||||
if (stacktraces.hasNext()) {
|
||||
cause = stacktraces.next();
|
||||
}
|
||||
crash(); // corrupt any unsynced-files
|
||||
if (LuceneTestCase.VERBOSE) {
|
||||
System.out.println("\nNOTE: MockDirectoryWrapper: now run CheckIndex");
|
||||
}
|
||||
TestUtil.checkIndex(this, getCrossCheckTermVectorsOnClose(), true);
|
||||
|
||||
// TODO: factor this out / share w/ TestIW.assertNoUnreferencedFiles
|
||||
if (assertNoUnreferencedFilesOnClose) {
|
||||
// now look for unreferenced files: discount ones that we tried to delete but could not
|
||||
Set<String> allFiles = new HashSet<>(Arrays.asList(listAll()));
|
||||
allFiles.removeAll(pendingDeletions);
|
||||
String[] startFiles = allFiles.toArray(new String[0]);
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(null);
|
||||
iwc.setIndexDeletionPolicy(NoDeletionPolicy.INSTANCE);
|
||||
new IndexWriter(in, iwc).rollback();
|
||||
String[] endFiles = in.listAll();
|
||||
|
||||
Set<String> startSet = new TreeSet<>(Arrays.asList(startFiles));
|
||||
Set<String> endSet = new TreeSet<>(Arrays.asList(endFiles));
|
||||
|
||||
if (pendingDeletions.contains("segments.gen") && endSet.contains("segments.gen")) {
|
||||
// this is possible if we hit an exception while writing segments.gen, we try to delete it
|
||||
// and it ends out in pendingDeletions (but IFD wont remove this).
|
||||
startSet.add("segments.gen");
|
||||
if (LuceneTestCase.VERBOSE) {
|
||||
System.out.println("MDW: Unreferenced check: Ignoring segments.gen that we could not delete.");
|
||||
}
|
||||
// RuntimeException instead of IOException because
|
||||
// super() does not throw IOException currently:
|
||||
throw new RuntimeException("MockDirectoryWrapper: cannot close: there are still open files: " + openFiles, cause);
|
||||
}
|
||||
if (openLocks.size() > 0) {
|
||||
Exception cause = null;
|
||||
Iterator<Exception> stacktraces = openLocks.values().iterator();
|
||||
if (stacktraces.hasNext()) {
|
||||
cause = stacktraces.next();
|
||||
}
|
||||
throw new RuntimeException("MockDirectoryWrapper: cannot close: there are still open locks: " + openLocks, cause);
|
||||
}
|
||||
|
||||
if (getCheckIndexOnClose()) {
|
||||
randomIOExceptionRate = 0.0;
|
||||
randomIOExceptionRateOnOpen = 0.0;
|
||||
if (DirectoryReader.indexExists(this)) {
|
||||
if (LuceneTestCase.VERBOSE) {
|
||||
System.out.println("\nNOTE: MockDirectoryWrapper: now crush");
|
||||
}
|
||||
crash(); // corrupt any unsynced-files
|
||||
if (LuceneTestCase.VERBOSE) {
|
||||
System.out.println("\nNOTE: MockDirectoryWrapper: now run CheckIndex");
|
||||
}
|
||||
TestUtil.checkIndex(this, getCrossCheckTermVectorsOnClose(), true);
|
||||
|
||||
// its possible we cannot delete the segments_N on windows if someone has it open and
|
||||
// maybe other files too, depending on timing. normally someone on windows wouldnt have
|
||||
// an issue (IFD would nuke this stuff eventually), but we pass NoDeletionPolicy...
|
||||
for (String file : pendingDeletions) {
|
||||
if (file.startsWith("segments") && !file.equals("segments.gen") && endSet.contains(file)) {
|
||||
startSet.add(file);
|
||||
// TODO: factor this out / share w/ TestIW.assertNoUnreferencedFiles
|
||||
if (assertNoUnreferencedFilesOnClose) {
|
||||
// now look for unreferenced files: discount ones that we tried to delete but could not
|
||||
Set<String> allFiles = new HashSet<>(Arrays.asList(listAll()));
|
||||
allFiles.removeAll(pendingDeletions);
|
||||
String[] startFiles = allFiles.toArray(new String[0]);
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(null);
|
||||
iwc.setIndexDeletionPolicy(NoDeletionPolicy.INSTANCE);
|
||||
new IndexWriter(in, iwc).rollback();
|
||||
String[] endFiles = in.listAll();
|
||||
|
||||
Set<String> startSet = new TreeSet<>(Arrays.asList(startFiles));
|
||||
Set<String> endSet = new TreeSet<>(Arrays.asList(endFiles));
|
||||
|
||||
if (pendingDeletions.contains("segments.gen") && endSet.contains("segments.gen")) {
|
||||
// this is possible if we hit an exception while writing segments.gen, we try to delete it
|
||||
// and it ends out in pendingDeletions (but IFD wont remove this).
|
||||
startSet.add("segments.gen");
|
||||
if (LuceneTestCase.VERBOSE) {
|
||||
System.out.println("MDW: Unreferenced check: Ignoring segments file: " + file + " that we could not delete.");
|
||||
System.out.println("MDW: Unreferenced check: Ignoring segments.gen that we could not delete.");
|
||||
}
|
||||
SegmentInfos sis;
|
||||
try {
|
||||
sis = SegmentInfos.readCommit(in, file);
|
||||
} catch (IOException ioe) {
|
||||
// OK: likely some of the .si files were deleted
|
||||
sis = new SegmentInfos();
|
||||
}
|
||||
|
||||
try {
|
||||
Set<String> ghosts = new HashSet<>(sis.files(in, false));
|
||||
for (String s : ghosts) {
|
||||
if (endSet.contains(s) && !startSet.contains(s)) {
|
||||
assert pendingDeletions.contains(s);
|
||||
if (LuceneTestCase.VERBOSE) {
|
||||
System.out.println("MDW: Unreferenced check: Ignoring referenced file: " + s + " " +
|
||||
"from " + file + " that we could not delete.");
|
||||
}
|
||||
startSet.add(s);
|
||||
}
|
||||
}
|
||||
|
||||
// its possible we cannot delete the segments_N on windows if someone has it open and
|
||||
// maybe other files too, depending on timing. normally someone on windows wouldnt have
|
||||
// an issue (IFD would nuke this stuff eventually), but we pass NoDeletionPolicy...
|
||||
for (String file : pendingDeletions) {
|
||||
if (file.startsWith("segments") && !file.equals("segments.gen") && endSet.contains(file)) {
|
||||
startSet.add(file);
|
||||
if (LuceneTestCase.VERBOSE) {
|
||||
System.out.println("MDW: Unreferenced check: Ignoring segments file: " + file + " that we could not delete.");
|
||||
}
|
||||
SegmentInfos sis;
|
||||
try {
|
||||
sis = SegmentInfos.readCommit(in, file);
|
||||
} catch (IOException ioe) {
|
||||
// OK: likely some of the .si files were deleted
|
||||
sis = new SegmentInfos();
|
||||
}
|
||||
|
||||
try {
|
||||
Set<String> ghosts = new HashSet<>(sis.files(in, false));
|
||||
for (String s : ghosts) {
|
||||
if (endSet.contains(s) && !startSet.contains(s)) {
|
||||
assert pendingDeletions.contains(s);
|
||||
if (LuceneTestCase.VERBOSE) {
|
||||
System.out.println("MDW: Unreferenced check: Ignoring referenced file: " + s + " " +
|
||||
"from " + file + " that we could not delete.");
|
||||
}
|
||||
startSet.add(s);
|
||||
}
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
System.err.println("ERROR processing leftover segments file " + file + ":");
|
||||
t.printStackTrace();
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
System.err.println("ERROR processing leftover segments file " + file + ":");
|
||||
t.printStackTrace();
|
||||
}
|
||||
}
|
||||
|
||||
startFiles = startSet.toArray(new String[0]);
|
||||
endFiles = endSet.toArray(new String[0]);
|
||||
|
||||
if (!Arrays.equals(startFiles, endFiles)) {
|
||||
List<String> removed = new ArrayList<>();
|
||||
for(String fileName : startFiles) {
|
||||
if (!endSet.contains(fileName)) {
|
||||
removed.add(fileName);
|
||||
}
|
||||
}
|
||||
|
||||
List<String> added = new ArrayList<>();
|
||||
for(String fileName : endFiles) {
|
||||
if (!startSet.contains(fileName)) {
|
||||
added.add(fileName);
|
||||
}
|
||||
}
|
||||
|
||||
String extras;
|
||||
if (removed.size() != 0) {
|
||||
extras = "\n\nThese files were removed: " + removed;
|
||||
} else {
|
||||
extras = "";
|
||||
}
|
||||
|
||||
if (added.size() != 0) {
|
||||
extras += "\n\nThese files were added (waaaaaaaaaat!): " + added;
|
||||
}
|
||||
|
||||
if (pendingDeletions.size() != 0) {
|
||||
extras += "\n\nThese files we had previously tried to delete, but couldn't: " + pendingDeletions;
|
||||
}
|
||||
|
||||
throw new RuntimeException("unreferenced files: before delete:\n " + Arrays.toString(startFiles) + "\n after delete:\n " + Arrays.toString(endFiles) + extras);
|
||||
}
|
||||
|
||||
DirectoryReader ir1 = DirectoryReader.open(this);
|
||||
int numDocs1 = ir1.numDocs();
|
||||
ir1.close();
|
||||
new IndexWriter(this, new IndexWriterConfig(null)).close();
|
||||
DirectoryReader ir2 = DirectoryReader.open(this);
|
||||
int numDocs2 = ir2.numDocs();
|
||||
ir2.close();
|
||||
assert numDocs1 == numDocs2 : "numDocs changed after opening/closing IW: before=" + numDocs1 + " after=" + numDocs2;
|
||||
}
|
||||
|
||||
startFiles = startSet.toArray(new String[0]);
|
||||
endFiles = endSet.toArray(new String[0]);
|
||||
|
||||
if (!Arrays.equals(startFiles, endFiles)) {
|
||||
List<String> removed = new ArrayList<>();
|
||||
for(String fileName : startFiles) {
|
||||
if (!endSet.contains(fileName)) {
|
||||
removed.add(fileName);
|
||||
}
|
||||
}
|
||||
|
||||
List<String> added = new ArrayList<>();
|
||||
for(String fileName : endFiles) {
|
||||
if (!startSet.contains(fileName)) {
|
||||
added.add(fileName);
|
||||
}
|
||||
}
|
||||
|
||||
String extras;
|
||||
if (removed.size() != 0) {
|
||||
extras = "\n\nThese files were removed: " + removed;
|
||||
} else {
|
||||
extras = "";
|
||||
}
|
||||
|
||||
if (added.size() != 0) {
|
||||
extras += "\n\nThese files were added (waaaaaaaaaat!): " + added;
|
||||
}
|
||||
|
||||
if (pendingDeletions.size() != 0) {
|
||||
extras += "\n\nThese files we had previously tried to delete, but couldn't: " + pendingDeletions;
|
||||
}
|
||||
|
||||
throw new RuntimeException("unreferenced files: before delete:\n " + Arrays.toString(startFiles) + "\n after delete:\n " + Arrays.toString(endFiles) + extras);
|
||||
}
|
||||
|
||||
DirectoryReader ir1 = DirectoryReader.open(this);
|
||||
int numDocs1 = ir1.numDocs();
|
||||
ir1.close();
|
||||
new IndexWriter(this, new IndexWriterConfig(null)).close();
|
||||
DirectoryReader ir2 = DirectoryReader.open(this);
|
||||
int numDocs2 = ir2.numDocs();
|
||||
ir2.close();
|
||||
assert numDocs1 == numDocs2 : "numDocs changed after opening/closing IW: before=" + numDocs1 + " after=" + numDocs2;
|
||||
}
|
||||
}
|
||||
success = true;
|
||||
} finally {
|
||||
if (success) {
|
||||
IOUtils.close(in);
|
||||
} else {
|
||||
IOUtils.closeWhileHandlingException(in);
|
||||
}
|
||||
}
|
||||
in.close();
|
||||
}
|
||||
|
||||
synchronized void removeOpenFile(Closeable c, String name) {
|
||||
|
|
Loading…
Reference in New Issue