mirror of https://github.com/apache/lucene.git
LUCENE-3601: fix missing incRef for _N.fnx; assert no unref'd files in MDW.close
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1207577 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
9a513f82dc
commit
c80bb6e229
|
@ -22,7 +22,6 @@ import java.io.IOException;
|
|||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.LinkedList;
|
||||
|
@ -52,7 +51,6 @@ import org.apache.lucene.util.BitVector;
|
|||
import org.apache.lucene.util.Constants;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.InfoStream;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
import org.apache.lucene.util.ThreadInterruptedException;
|
||||
import org.apache.lucene.util.MapBackedSet;
|
||||
import org.apache.lucene.util.TwoPhaseCommit;
|
||||
|
@ -212,6 +210,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
volatile SegmentInfos pendingCommit; // set when a commit is pending (after prepareCommit() & before commit())
|
||||
volatile long pendingCommitChangeCount;
|
||||
|
||||
private Collection<String> filesToCommit;
|
||||
|
||||
final SegmentInfos segmentInfos; // the segments
|
||||
final FieldNumberBiMap globalFieldNumberMap;
|
||||
|
||||
|
@ -2654,8 +2654,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
// to commit. This is important in case, eg, while
|
||||
// we are trying to sync all referenced files, a
|
||||
// merge completes which would otherwise have
|
||||
// removed the files we are now syncing.
|
||||
deleter.incRef(toCommit, false);
|
||||
// removed the files we are now syncing.
|
||||
filesToCommit = toCommit.files(directory, false);
|
||||
deleter.incRef(filesToCommit);
|
||||
}
|
||||
success = true;
|
||||
} finally {
|
||||
|
@ -2680,7 +2681,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
} finally {
|
||||
if (!success) {
|
||||
synchronized (this) {
|
||||
deleter.decRef(toCommit);
|
||||
deleter.decRef(filesToCommit);
|
||||
filesToCommit = null;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -2780,8 +2782,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
rollbackSegments = pendingCommit.createBackupSegmentInfos(true);
|
||||
deleter.checkpoint(pendingCommit, true);
|
||||
} finally {
|
||||
// Matches the incRef done in startCommit:
|
||||
deleter.decRef(pendingCommit);
|
||||
// Matches the incRef done in prepareCommit:
|
||||
deleter.decRef(filesToCommit);
|
||||
filesToCommit = null;
|
||||
pendingCommit = null;
|
||||
notifyAll();
|
||||
}
|
||||
|
@ -3796,7 +3799,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
if (infoStream != null) {
|
||||
infoStream.message("IW", " skip startCommit(): no changes pending");
|
||||
}
|
||||
deleter.decRef(toSync);
|
||||
deleter.decRef(filesToCommit);
|
||||
filesToCommit = null;
|
||||
return;
|
||||
}
|
||||
|
||||
|
@ -3857,7 +3861,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
}
|
||||
|
||||
// Hit exception
|
||||
deleter.decRef(toSync);
|
||||
deleter.decRef(filesToCommit);
|
||||
filesToCommit = null;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -875,9 +875,9 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfo> {
|
|||
*/
|
||||
files.add(segmentFileName);
|
||||
}
|
||||
if (lastGlobalFieldMapVersion > 0) {
|
||||
files.add(getGlobalFieldNumberName(lastGlobalFieldMapVersion));
|
||||
}
|
||||
}
|
||||
if (lastGlobalFieldMapVersion > 0) {
|
||||
files.add(getGlobalFieldNumberName(lastGlobalFieldMapVersion));
|
||||
}
|
||||
final int size = size();
|
||||
for(int i=0;i<size;i++) {
|
||||
|
|
|
@ -34,6 +34,8 @@ import java.util.Set;
|
|||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexWriter;
|
||||
import org.apache.lucene.index.IndexWriterConfig;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.ThrottledIndexOutput;
|
||||
import org.apache.lucene.util._TestUtil;
|
||||
|
@ -494,6 +496,12 @@ public class MockDirectoryWrapper extends Directory {
|
|||
return size;
|
||||
}
|
||||
|
||||
private boolean assertNoUnreferencedFilesOnClose = true;
|
||||
|
||||
public void setAssertNoUnrefencedFilesOnClose(boolean v) {
|
||||
assertNoUnreferencedFilesOnClose = v;
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void close() throws IOException {
|
||||
maybeYield();
|
||||
|
@ -526,6 +534,20 @@ public class MockDirectoryWrapper extends Directory {
|
|||
System.out.println("\nNOTE: MockDirectoryWrapper: now run CheckIndex");
|
||||
}
|
||||
_TestUtil.checkIndex(this);
|
||||
|
||||
if (assertNoUnreferencedFilesOnClose) {
|
||||
// now look for unreferenced files:
|
||||
String[] startFiles = listAll();
|
||||
new IndexWriter(this, new IndexWriterConfig(LuceneTestCase.TEST_VERSION_CURRENT, null)).rollback();
|
||||
String[] endFiles = listAll();
|
||||
|
||||
Arrays.sort(startFiles);
|
||||
Arrays.sort(endFiles);
|
||||
|
||||
if (!Arrays.equals(startFiles, endFiles)) {
|
||||
assert false : "unreferenced files: before delete:\n " + Arrays.toString(startFiles) + "\n after delete:\n " + Arrays.toString(endFiles);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
delegate.close();
|
||||
|
|
|
@ -75,7 +75,6 @@ public class TestGlobalFieldNumbers extends LuceneTestCase {
|
|||
for (String string : files) {
|
||||
assertFalse(string.endsWith(".fnx"));
|
||||
}
|
||||
assertFNXFiles(dir, "_2.fnx");
|
||||
writer.close();
|
||||
assertFNXFiles(dir, "_2.fnx");
|
||||
}
|
||||
|
@ -127,7 +126,6 @@ public class TestGlobalFieldNumbers extends LuceneTestCase {
|
|||
d.add(new BinaryField("f3", new byte[] { 1, 2, 3 }));
|
||||
writer.addDocument(d);
|
||||
writer.commit();
|
||||
assertFNXFiles(dir, "_2.fnx");
|
||||
writer.close();
|
||||
assertFNXFiles(dir, "_2.fnx");
|
||||
}
|
||||
|
|
|
@ -43,7 +43,6 @@ import org.apache.lucene.document.StringField;
|
|||
import org.apache.lucene.document.TextField;
|
||||
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.DefaultTermVectorsReader;
|
||||
import org.apache.lucene.index.codecs.simpletext.SimpleTextCodec;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.FieldCache;
|
||||
|
@ -149,8 +148,6 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
|
||||
public static void assertNoUnreferencedFiles(Directory dir, String message) throws IOException {
|
||||
String[] startFiles = dir.listAll();
|
||||
SegmentInfos infos = new SegmentInfos();
|
||||
infos.read(dir);
|
||||
new IndexWriter(dir, new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random))).rollback();
|
||||
String[] endFiles = dir.listAll();
|
||||
|
||||
|
|
|
@ -21,6 +21,7 @@ import java.io.File;
|
|||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.MockDirectoryWrapper;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
|
||||
/**
|
||||
|
@ -30,6 +31,12 @@ public class MockDirectoryFactory extends CachingDirectoryFactory {
|
|||
|
||||
@Override
|
||||
public Directory create(String path) throws IOException {
|
||||
return LuceneTestCase.newFSDirectory(new File(path));
|
||||
MockDirectoryWrapper dir = LuceneTestCase.newFSDirectory(new File(path));
|
||||
// Somehow removing unref'd files in Solr tests causes
|
||||
// problems... there's some interaction w/
|
||||
// CachingDirectoryFactory. Once we track down where Solr
|
||||
// isn't closing an IW, we can re-enable this:
|
||||
dir.setAssertNoUnrefencedFilesOnClose(false);
|
||||
return dir;
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue