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
|
@ -2068,15 +2068,15 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
* get in our way and do unnecessary work. -- if we don't lock this here we might
|
* get in our way and do unnecessary work. -- if we don't lock this here we might
|
||||||
* get in trouble if */
|
* get in trouble if */
|
||||||
synchronized (fullFlushLock) {
|
synchronized (fullFlushLock) {
|
||||||
/*
|
/*
|
||||||
* We first abort and trash everything we have in-memory
|
* We first abort and trash everything we have in-memory
|
||||||
* and keep the thread-states locked, the lockAndAbortAll operation
|
* and keep the thread-states locked, the lockAndAbortAll operation
|
||||||
* also guarantees "point in time semantics" ie. the checkpoint that we need in terms
|
* also guarantees "point in time semantics" ie. the checkpoint that we need in terms
|
||||||
* of logical happens-before relationship in the DW. So we do
|
* of logical happens-before relationship in the DW. So we do
|
||||||
* abort all in memory structures
|
* abort all in memory structures
|
||||||
* We also drop global field numbering before during abort to make
|
* We also drop global field numbering before during abort to make
|
||||||
* sure it's just like a fresh index.
|
* sure it's just like a fresh index.
|
||||||
*/
|
*/
|
||||||
try {
|
try {
|
||||||
docWriter.lockAndAbortAll(this);
|
docWriter.lockAndAbortAll(this);
|
||||||
processEvents(false, true);
|
processEvents(false, true);
|
||||||
|
@ -2101,8 +2101,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
segmentInfos.changed();
|
segmentInfos.changed();
|
||||||
globalFieldNumberMap.clear();
|
globalFieldNumberMap.clear();
|
||||||
success = true;
|
success = true;
|
||||||
} catch (OutOfMemoryError oom) {
|
|
||||||
tragicEvent(oom, "deleteAll");
|
|
||||||
} finally {
|
} finally {
|
||||||
if (!success) {
|
if (!success) {
|
||||||
if (infoStream.isEnabled("IW")) {
|
if (infoStream.isEnabled("IW")) {
|
||||||
|
@ -2111,6 +2109,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
} catch (OutOfMemoryError oom) {
|
||||||
|
tragicEvent(oom, "deleteAll");
|
||||||
} finally {
|
} finally {
|
||||||
docWriter.unlockAllAfterAbortAll(this);
|
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;
|
||||||
|
|
||||||
if (pendingCommit != null) {
|
try {
|
||||||
try {
|
synchronized(this) {
|
||||||
if (infoStream.isEnabled("IW")) {
|
if (pendingCommit != null) {
|
||||||
infoStream.message("IW", "commit: pendingCommit != null");
|
try {
|
||||||
}
|
|
||||||
pendingCommit.finishCommit(directory);
|
if (infoStream.isEnabled("IW")) {
|
||||||
success = true;
|
infoStream.message("IW", "commit: pendingCommit != null");
|
||||||
// we committed, if anything goes wrong after this: we are screwed
|
}
|
||||||
try {
|
|
||||||
if (infoStream.isEnabled("IW")) {
|
committedSegmentsFileName = pendingCommit.finishCommit(directory);
|
||||||
infoStream.message("IW", "commit: wrote segments file \"" + pendingCommit.getSegmentsFileName() + "\"");
|
|
||||||
}
|
// we committed, if anything goes wrong after this, we are screwed and it's a tragedy:
|
||||||
segmentInfos.updateGeneration(pendingCommit);
|
commitCompleted = true;
|
||||||
lastCommitChangeCount = pendingCommitChangeCount;
|
|
||||||
rollbackSegments = pendingCommit.createBackupSegmentInfos();
|
// NOTE: don't use this.checkpoint() here, because
|
||||||
// NOTE: don't use this.checkpoint() here, because
|
// we do not want to increment changeCount:
|
||||||
// we do not want to increment changeCount:
|
deleter.checkpoint(pendingCommit, true);
|
||||||
deleter.checkpoint(pendingCommit, true);
|
|
||||||
} catch (Throwable tragedy) {
|
lastCommitChangeCount = pendingCommitChangeCount;
|
||||||
tragicEvent(tragedy, "finishCommit");
|
rollbackSegments = pendingCommit.createBackupSegmentInfos();
|
||||||
}
|
|
||||||
} finally {
|
finished = true;
|
||||||
// Matches the incRef done in prepareCommit:
|
} finally {
|
||||||
try {
|
notifyAll();
|
||||||
if (success == false || tragedy == null) {
|
|
||||||
try {
|
try {
|
||||||
deleter.decRef(filesToCommit);
|
if (finished) {
|
||||||
} catch (Throwable t) {
|
// all is good
|
||||||
// if the commit succeeded, we are in screwed state
|
deleter.decRef(filesToCommit);
|
||||||
// otherwise, throw our original exception
|
} else if (commitCompleted == false) {
|
||||||
if (success) {
|
// exc happened in finishCommit: not a tragedy
|
||||||
tragicEvent(tragedy, "finishCommit");
|
deleter.decRefWhileHandlingException(filesToCommit);
|
||||||
}
|
}
|
||||||
|
} finally {
|
||||||
|
pendingCommit = null;
|
||||||
|
filesToCommit = null;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
} finally {
|
} else {
|
||||||
filesToCommit = null;
|
assert filesToCommit == null;
|
||||||
pendingCommit = null;
|
if (infoStream.isEnabled("IW")) {
|
||||||
notifyAll();
|
infoStream.message("IW", "commit: pendingCommit == null; skip");
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
} catch (Throwable t) {
|
||||||
if (infoStream.isEnabled("IW")) {
|
if (infoStream.isEnabled("IW")) {
|
||||||
infoStream.message("IW", String.format(Locale.ROOT, "commit: took %.1f msec", (System.nanoTime()-startCommitTime)/1000000.0));
|
infoStream.message("IW", "hit exception during finishCommit: " + t.getMessage());
|
||||||
}
|
}
|
||||||
|
if (commitCompleted) {
|
||||||
} else {
|
tragicEvent(t, "finishCommit");
|
||||||
if (infoStream.isEnabled("IW")) {
|
} else {
|
||||||
infoStream.message("IW", "commit: pendingCommit == null; skip");
|
IOUtils.reThrow(t);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if (infoStream.isEnabled("IW")) {
|
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");
|
infoStream.message("IW", "commit: done");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -4375,12 +4382,17 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
}
|
}
|
||||||
|
|
||||||
private void tragicEvent(Throwable tragedy, String location) {
|
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")) {
|
if (infoStream.isEnabled("IW")) {
|
||||||
infoStream.message("IW", "hit " + tragedy.getClass().getSimpleName() + " inside " + location);
|
infoStream.message("IW", "hit " + tragedy.getClass().getSimpleName() + " inside " + location);
|
||||||
}
|
}
|
||||||
// its possible you could have a really bad day
|
synchronized (this) {
|
||||||
if (this.tragedy == null) {
|
// its possible you could have a really bad day
|
||||||
this.tragedy = tragedy;
|
if (this.tragedy == null) {
|
||||||
|
this.tragedy = tragedy;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
// if we are already closed (e.g. called by rollback), this will be a no-op.
|
// if we are already closed (e.g. called by rollback), this will be a no-op.
|
||||||
synchronized(commitLock) {
|
synchronized(commitLock) {
|
||||||
|
|
|
@ -704,14 +704,16 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
|
||||||
return files;
|
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) {
|
if (pendingCommit == false) {
|
||||||
throw new IllegalStateException("prepareCommit was not called");
|
throw new IllegalStateException("prepareCommit was not called");
|
||||||
}
|
}
|
||||||
boolean success = false;
|
boolean success = false;
|
||||||
|
final String dest;
|
||||||
try {
|
try {
|
||||||
final String src = IndexFileNames.fileNameFromGeneration(IndexFileNames.PENDING_SEGMENTS, "", generation);
|
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);
|
dir.renameFile(src, dest);
|
||||||
success = true;
|
success = true;
|
||||||
} finally {
|
} finally {
|
||||||
|
@ -723,6 +725,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
|
||||||
|
|
||||||
pendingCommit = false;
|
pendingCommit = false;
|
||||||
lastGeneration = generation;
|
lastGeneration = generation;
|
||||||
|
return dest;
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Writes & syncs to the Directory dir, taking care to
|
/** Writes & syncs to the Directory dir, taking care to
|
||||||
|
|
Loading…
Reference in New Issue