mirror of https://github.com/apache/lucene.git
LUCENE-5958: fix IW deadlock
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1627003 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
1dd2ecb282
commit
da87077edd
|
@ -2101,8 +2101,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
segmentInfos.changed();
|
||||
globalFieldNumberMap.clear();
|
||||
success = true;
|
||||
} catch (OutOfMemoryError oom) {
|
||||
tragicEvent(oom, "deleteAll");
|
||||
} finally {
|
||||
if (!success) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
|
@ -2111,6 +2109,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
}
|
||||
}
|
||||
}
|
||||
} catch (OutOfMemoryError oom) {
|
||||
tragicEvent(oom, "deleteAll");
|
||||
} finally {
|
||||
docWriter.unlockAllAfterAbortAll(this);
|
||||
}
|
||||
|
@ -2889,63 +2889,70 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
}
|
||||
}
|
||||
|
||||
private synchronized final void finishCommit() throws IOException {
|
||||
private final void finishCommit() throws IOException {
|
||||
|
||||
boolean success = false;
|
||||
boolean commitCompleted = false;
|
||||
boolean finished = false;
|
||||
String committedSegmentsFileName = null;
|
||||
|
||||
try {
|
||||
synchronized(this) {
|
||||
if (pendingCommit != null) {
|
||||
try {
|
||||
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "commit: pendingCommit != null");
|
||||
}
|
||||
pendingCommit.finishCommit(directory);
|
||||
success = true;
|
||||
// we committed, if anything goes wrong after this: we are screwed
|
||||
try {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "commit: wrote segments file \"" + pendingCommit.getSegmentsFileName() + "\"");
|
||||
}
|
||||
segmentInfos.updateGeneration(pendingCommit);
|
||||
lastCommitChangeCount = pendingCommitChangeCount;
|
||||
rollbackSegments = pendingCommit.createBackupSegmentInfos();
|
||||
|
||||
committedSegmentsFileName = pendingCommit.finishCommit(directory);
|
||||
|
||||
// we committed, if anything goes wrong after this, we are screwed and it's a tragedy:
|
||||
commitCompleted = true;
|
||||
|
||||
// NOTE: don't use this.checkpoint() here, because
|
||||
// we do not want to increment changeCount:
|
||||
deleter.checkpoint(pendingCommit, true);
|
||||
} catch (Throwable tragedy) {
|
||||
tragicEvent(tragedy, "finishCommit");
|
||||
}
|
||||
|
||||
lastCommitChangeCount = pendingCommitChangeCount;
|
||||
rollbackSegments = pendingCommit.createBackupSegmentInfos();
|
||||
|
||||
finished = true;
|
||||
} finally {
|
||||
// Matches the incRef done in prepareCommit:
|
||||
try {
|
||||
if (success == false || tragedy == null) {
|
||||
try {
|
||||
deleter.decRef(filesToCommit);
|
||||
} catch (Throwable t) {
|
||||
// if the commit succeeded, we are in screwed state
|
||||
// otherwise, throw our original exception
|
||||
if (success) {
|
||||
tragicEvent(tragedy, "finishCommit");
|
||||
}
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
filesToCommit = null;
|
||||
pendingCommit = null;
|
||||
notifyAll();
|
||||
try {
|
||||
if (finished) {
|
||||
// all is good
|
||||
deleter.decRef(filesToCommit);
|
||||
} else if (commitCompleted == false) {
|
||||
// exc happened in finishCommit: not a tragedy
|
||||
deleter.decRefWhileHandlingException(filesToCommit);
|
||||
}
|
||||
} finally {
|
||||
pendingCommit = null;
|
||||
filesToCommit = null;
|
||||
}
|
||||
}
|
||||
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", String.format(Locale.ROOT, "commit: took %.1f msec", (System.nanoTime()-startCommitTime)/1000000.0));
|
||||
}
|
||||
|
||||
} else {
|
||||
assert filesToCommit == null;
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "commit: pendingCommit == null; skip");
|
||||
}
|
||||
}
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "hit exception during finishCommit: " + t.getMessage());
|
||||
}
|
||||
if (commitCompleted) {
|
||||
tragicEvent(t, "finishCommit");
|
||||
} else {
|
||||
IOUtils.reThrow(t);
|
||||
}
|
||||
}
|
||||
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "commit: wrote segments file \"" + committedSegmentsFileName + "\"");
|
||||
infoStream.message("IW", String.format(Locale.ROOT, "commit: took %.1f msec", (System.nanoTime()-startCommitTime)/1000000.0));
|
||||
infoStream.message("IW", "commit: done");
|
||||
}
|
||||
}
|
||||
|
@ -4375,13 +4382,18 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
}
|
||||
|
||||
private void tragicEvent(Throwable tragedy, String location) {
|
||||
// We cannot hold IW's lock here else it can lead to deadlock:
|
||||
assert Thread.holdsLock(this) == false;
|
||||
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "hit " + tragedy.getClass().getSimpleName() + " inside " + location);
|
||||
}
|
||||
synchronized (this) {
|
||||
// its possible you could have a really bad day
|
||||
if (this.tragedy == null) {
|
||||
this.tragedy = tragedy;
|
||||
}
|
||||
}
|
||||
// if we are already closed (e.g. called by rollback), this will be a no-op.
|
||||
synchronized(commitLock) {
|
||||
if (closing == false) {
|
||||
|
|
|
@ -704,14 +704,16 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
|
|||
return files;
|
||||
}
|
||||
|
||||
final void finishCommit(Directory dir) throws IOException {
|
||||
/** Returns the committed segments_N filename. */
|
||||
final String finishCommit(Directory dir) throws IOException {
|
||||
if (pendingCommit == false) {
|
||||
throw new IllegalStateException("prepareCommit was not called");
|
||||
}
|
||||
boolean success = false;
|
||||
final String dest;
|
||||
try {
|
||||
final String src = IndexFileNames.fileNameFromGeneration(IndexFileNames.PENDING_SEGMENTS, "", generation);
|
||||
final String dest = IndexFileNames.fileNameFromGeneration(IndexFileNames.SEGMENTS, "", generation);
|
||||
dest = IndexFileNames.fileNameFromGeneration(IndexFileNames.SEGMENTS, "", generation);
|
||||
dir.renameFile(src, dest);
|
||||
success = true;
|
||||
} finally {
|
||||
|
@ -723,6 +725,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
|
|||
|
||||
pendingCommit = false;
|
||||
lastGeneration = generation;
|
||||
return dest;
|
||||
}
|
||||
|
||||
/** Writes & syncs to the Directory dir, taking care to
|
||||
|
|
Loading…
Reference in New Issue