LUCENE-5919: fix IndexWriter exception handling when calling IndexFileDeleter.decRef

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1622279 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael McCandless 2014-09-03 15:26:03 +00:00
parent fbb8dc76e3
commit c84af3d08a
4 changed files with 171 additions and 24 deletions

View File

@ -136,6 +136,11 @@ Bug Fixes
tests (and test iterations). Fix for FaultyIndexInput in particular.
(Dawid Weiss)
* LUCENE-5919: Fix exception handling inside IndexWriter when
deleteFile throws an exception, to not over-decRef index files,
possibly deleting a file that's still in use in the index, leading
to corruption. (Mike McCandless)
Build
* LUCENE-5909: Smoke tester now has better command line parsing and

View File

@ -33,6 +33,7 @@ import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.NoSuchDirectoryException;
import org.apache.lucene.util.CollectionUtil;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.InfoStream;
/*
@ -367,17 +368,25 @@ final class IndexFileDeleter implements Closeable {
// First decref all files that had been referred to by
// the now-deleted commits:
Throwable firstThrowable = null;
for(int i=0;i<size;i++) {
CommitPoint commit = commitsToDelete.get(i);
if (infoStream.isEnabled("IFD")) {
infoStream.message("IFD", "deleteCommits: now decRef commit \"" + commit.getSegmentsFileName() + "\"");
}
for (final String file : commit.files) {
decRef(file);
try {
decRef(commit.files);
} catch (Throwable t) {
if (firstThrowable == null) {
firstThrowable = t;
}
}
}
commitsToDelete.clear();
// NOTE: does nothing if firstThrowable is null
IOUtils.reThrow(firstThrowable);
// Now compact commits to remove deleted ones (preserving the sort):
size = commits.size();
int readFrom = 0;
@ -456,8 +465,11 @@ final class IndexFileDeleter implements Closeable {
assert locked();
if (!lastFiles.isEmpty()) {
decRef(lastFiles);
lastFiles.clear();
try {
decRef(lastFiles);
} finally {
lastFiles.clear();
}
}
deletePendingFiles();
@ -554,8 +566,11 @@ final class IndexFileDeleter implements Closeable {
deleteCommits();
} else {
// DecRef old files from the last checkpoint, if any:
decRef(lastFiles);
lastFiles.clear();
try {
decRef(lastFiles);
} finally {
lastFiles.clear();
}
// Save files so we can decr on next checkpoint/commit:
lastFiles.addAll(segmentInfos.files(directory, false));
@ -593,10 +608,34 @@ final class IndexFileDeleter implements Closeable {
rc.IncRef();
}
/** Decrefs all provided files, even on exception; throws first exception hit, if any. */
void decRef(Collection<String> files) throws IOException {
assert locked();
Throwable firstThrowable = null;
for(final String file : files) {
decRef(file);
try {
decRef(file);
} catch (Throwable t) {
if (firstThrowable == null) {
// Save first exception and throw it in the end, but be sure to finish decRef all files
firstThrowable = t;
}
}
}
// NOTE: does nothing if firstThrowable is null
IOUtils.reThrow(firstThrowable);
}
/** Decrefs all provided files, ignoring any exceptions hit; call this if
* you are already handling an exception. */
void decRefWhileHandlingException(Collection<String> files) throws IOException {
assert locked();
for(final String file : files) {
try {
decRef(file);
} catch (Throwable t) {
}
}
}
@ -611,16 +650,17 @@ final class IndexFileDeleter implements Closeable {
if (0 == rc.DecRef()) {
// This file is no longer referenced by any past
// commit points nor by the in-memory SegmentInfos:
deleteFile(fileName);
refCounts.remove(fileName);
try {
deleteFile(fileName);
} finally {
refCounts.remove(fileName);
}
}
}
void decRef(SegmentInfos segmentInfos) throws IOException {
assert locked();
for (final String file : segmentInfos.files(directory, false)) {
decRef(file);
}
decRef(segmentInfos.files(directory, false));
}
public boolean exists(String fileName) {

View File

@ -2019,9 +2019,12 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
if (pendingCommit != null) {
pendingCommit.rollbackCommit(directory);
deleter.decRef(pendingCommit);
pendingCommit = null;
notifyAll();
try {
deleter.decRef(pendingCommit);
} finally {
pendingCommit = null;
notifyAll();
}
}
// Don't bother saving any changes in our segmentInfos
@ -2074,7 +2077,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
try {
pendingCommit.rollbackCommit(directory);
deleter.decRef(pendingCommit);
} catch (Throwable t) {}
} catch (Throwable t) {
}
pendingCommit = null;
}
// close all the closeables we can (but important is readerPool and writeLock to prevent leaks)
@ -2848,7 +2853,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
if (!success) {
synchronized (this) {
if (filesToCommit != null) {
deleter.decRef(filesToCommit);
deleter.decRefWhileHandlingException(filesToCommit);
filesToCommit = null;
}
}
@ -2979,10 +2984,13 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
deleter.checkpoint(pendingCommit, true);
} finally {
// Matches the incRef done in prepareCommit:
deleter.decRef(filesToCommit);
filesToCommit = null;
pendingCommit = null;
notifyAll();
try {
deleter.decRef(filesToCommit);
} finally {
filesToCommit = null;
pendingCommit = null;
notifyAll();
}
}
if (infoStream.isEnabled("IW")) {
@ -4291,8 +4299,11 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", " skip startCommit(): no changes pending");
}
deleter.decRef(filesToCommit);
filesToCommit = null;
try {
deleter.decRef(filesToCommit);
} finally {
filesToCommit = null;
}
return;
}
@ -4363,7 +4374,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
}
// Hit exception
deleter.decRef(filesToCommit);
deleter.decRefWhileHandlingException(filesToCommit);
filesToCommit = null;
}
}

View File

@ -398,4 +398,95 @@ public class TestIndexFileDeleter extends LuceneTestCase {
dir.close();
}
// LUCENE-5919
public void testExcInDecRef() throws Exception {
MockDirectoryWrapper dir = newMockDirectory();
// disable slow things: we don't rely upon sleeps here.
dir.setThrottling(MockDirectoryWrapper.Throttling.NEVER);
dir.setUseSlowOpenClosers(false);
final AtomicBoolean doFailExc = new AtomicBoolean();
dir.failOn(new MockDirectoryWrapper.Failure() {
@Override
public void eval(MockDirectoryWrapper dir) throws IOException {
if (doFailExc.get() && random().nextInt(4) == 1) {
Exception e = new Exception();
StackTraceElement stack[] = e.getStackTrace();
for (int i = 0; i < stack.length; i++) {
if (stack[i].getClassName().equals(IndexFileDeleter.class.getName()) && stack[i].getMethodName().equals("decRef")) {
throw new RuntimeException("fake fail");
}
}
}
}
});
IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random()));
//iwc.setMergeScheduler(new SerialMergeScheduler());
MergeScheduler ms = iwc.getMergeScheduler();
if (ms instanceof ConcurrentMergeScheduler) {
final ConcurrentMergeScheduler suppressFakeFail = new ConcurrentMergeScheduler() {
@Override
protected void handleMergeException(Throwable exc) {
// suppress only FakeIOException:
if (exc instanceof RuntimeException && exc.getMessage().equals("fake fail")) {
// ok to ignore
} else {
super.handleMergeException(exc);
}
}
};
final ConcurrentMergeScheduler cms = (ConcurrentMergeScheduler) ms;
suppressFakeFail.setMaxMergesAndThreads(cms.getMaxMergeCount(), cms.getMaxThreadCount());
suppressFakeFail.setMergeThreadPriority(cms.getMergeThreadPriority());
iwc.setMergeScheduler(suppressFakeFail);
}
RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
// Since we hit exc during merging, a partial
// forceMerge can easily return when there are still
// too many segments in the index:
w.setDoRandomForceMergeAssert(false);
doFailExc.set(true);
int ITERS = atLeast(1000);
for(int iter=0;iter<ITERS;iter++) {
try {
if (random().nextInt(10) == 5) {
w.commit();
} else if (random().nextInt(10) == 7) {
w.getReader().close();
} else {
Document doc = new Document();
doc.add(newTextField("field", "some text", Field.Store.NO));
w.addDocument(doc);
}
} catch (IOException ioe) {
if (ioe.getMessage().contains("background merge hit exception")) {
Throwable cause = ioe.getCause();
if (cause != null && cause instanceof RuntimeException && ((RuntimeException) cause).getMessage().equals("fake fail")) {
// ok
} else {
throw ioe;
}
} else {
throw ioe;
}
} catch (RuntimeException re) {
if (re.getMessage().equals("fake fail")) {
// ok
} else {
throw re;
}
}
}
doFailExc.set(false);
w.close();
dir.close();
}
}