mirror of https://github.com/apache/lucene.git
LUCENE-7570: don't run merges while holding the commitLock to prevent deadlock when merges are stalled and a tragic merge exception strikes
This commit is contained in:
parent
7cffae3a16
commit
2b073a2f29
|
@ -94,6 +94,10 @@ Bug Fixes
|
||||||
in the index sort, since this would lead to corruption. (Jim
|
in the index sort, since this would lead to corruption. (Jim
|
||||||
Ferenczi via Mike McCandless)
|
Ferenczi via Mike McCandless)
|
||||||
|
|
||||||
|
* LUCENE-7570: IndexWriter may deadlock if a commit is running while
|
||||||
|
there are too many merges running and one of the merges hits a
|
||||||
|
tragic exception (Joey Echeverria via Mike McCandless)
|
||||||
|
|
||||||
Improvements
|
Improvements
|
||||||
|
|
||||||
* LUCENE-6824: TermAutomatonQuery now rewrites to TermQuery,
|
* LUCENE-6824: TermAutomatonQuery now rewrites to TermQuery,
|
||||||
|
|
|
@ -2952,11 +2952,16 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
@Override
|
@Override
|
||||||
public final long prepareCommit() throws IOException {
|
public final long prepareCommit() throws IOException {
|
||||||
ensureOpen();
|
ensureOpen();
|
||||||
pendingSeqNo = prepareCommitInternal(config.getMergePolicy());
|
boolean[] doMaybeMerge = new boolean[1];
|
||||||
|
pendingSeqNo = prepareCommitInternal(doMaybeMerge);
|
||||||
|
// we must do this outside of the commitLock else we can deadlock:
|
||||||
|
if (doMaybeMerge[0]) {
|
||||||
|
maybeMerge(config.getMergePolicy(), MergeTrigger.FULL_FLUSH, UNBOUNDED_MAX_MERGE_SEGMENTS);
|
||||||
|
}
|
||||||
return pendingSeqNo;
|
return pendingSeqNo;
|
||||||
}
|
}
|
||||||
|
|
||||||
private long prepareCommitInternal(MergePolicy mergePolicy) throws IOException {
|
private long prepareCommitInternal(boolean[] doMaybeMerge) throws IOException {
|
||||||
startCommitTime = System.nanoTime();
|
startCommitTime = System.nanoTime();
|
||||||
synchronized(commitLock) {
|
synchronized(commitLock) {
|
||||||
ensureOpen(false);
|
ensureOpen(false);
|
||||||
|
@ -3063,7 +3068,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
boolean success = false;
|
boolean success = false;
|
||||||
try {
|
try {
|
||||||
if (anySegmentsFlushed) {
|
if (anySegmentsFlushed) {
|
||||||
maybeMerge(mergePolicy, MergeTrigger.FULL_FLUSH, UNBOUNDED_MAX_MERGE_SEGMENTS);
|
doMaybeMerge[0] = true;
|
||||||
}
|
}
|
||||||
startCommit(toCommit);
|
startCommit(toCommit);
|
||||||
success = true;
|
success = true;
|
||||||
|
@ -3184,6 +3189,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
infoStream.message("IW", "commit: start");
|
infoStream.message("IW", "commit: start");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
boolean[] doMaybeMerge = new boolean[1];
|
||||||
|
|
||||||
|
long seqNo;
|
||||||
|
|
||||||
synchronized(commitLock) {
|
synchronized(commitLock) {
|
||||||
ensureOpen(false);
|
ensureOpen(false);
|
||||||
|
|
||||||
|
@ -3191,13 +3200,11 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
infoStream.message("IW", "commit: enter lock");
|
infoStream.message("IW", "commit: enter lock");
|
||||||
}
|
}
|
||||||
|
|
||||||
long seqNo;
|
|
||||||
|
|
||||||
if (pendingCommit == null) {
|
if (pendingCommit == null) {
|
||||||
if (infoStream.isEnabled("IW")) {
|
if (infoStream.isEnabled("IW")) {
|
||||||
infoStream.message("IW", "commit: now prepare");
|
infoStream.message("IW", "commit: now prepare");
|
||||||
}
|
}
|
||||||
seqNo = prepareCommitInternal(mergePolicy);
|
seqNo = prepareCommitInternal(doMaybeMerge);
|
||||||
} else {
|
} else {
|
||||||
if (infoStream.isEnabled("IW")) {
|
if (infoStream.isEnabled("IW")) {
|
||||||
infoStream.message("IW", "commit: already prepared");
|
infoStream.message("IW", "commit: already prepared");
|
||||||
|
@ -3206,9 +3213,14 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
}
|
}
|
||||||
|
|
||||||
finishCommit();
|
finishCommit();
|
||||||
|
|
||||||
return seqNo;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// we must do this outside of the commitLock else we can deadlock:
|
||||||
|
if (doMaybeMerge[0]) {
|
||||||
|
maybeMerge(mergePolicy, MergeTrigger.FULL_FLUSH, UNBOUNDED_MAX_MERGE_SEGMENTS);
|
||||||
|
}
|
||||||
|
|
||||||
|
return seqNo;
|
||||||
}
|
}
|
||||||
|
|
||||||
private final void finishCommit() throws IOException {
|
private final void finishCommit() throws IOException {
|
||||||
|
|
|
@ -14,13 +14,15 @@
|
||||||
* See the License for the specific language governing permissions and
|
* See the License for the specific language governing permissions and
|
||||||
* limitations under the License.
|
* limitations under the License.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
package org.apache.lucene.index;
|
package org.apache.lucene.index;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
import java.util.concurrent.CountDownLatch;
|
import java.util.concurrent.CountDownLatch;
|
||||||
import java.util.concurrent.atomic.AtomicBoolean;
|
import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
|
|
||||||
import org.apache.lucene.document.Document;
|
import org.apache.lucene.document.Document;
|
||||||
|
import org.apache.lucene.store.Directory;
|
||||||
import org.apache.lucene.store.MockDirectoryWrapper;
|
import org.apache.lucene.store.MockDirectoryWrapper;
|
||||||
import org.apache.lucene.util.LuceneTestCase;
|
import org.apache.lucene.util.LuceneTestCase;
|
||||||
|
|
||||||
|
@ -92,4 +94,69 @@ public class TestTragicIndexWriterDeadlock extends LuceneTestCase {
|
||||||
w.close();
|
w.close();
|
||||||
dir.close();
|
dir.close();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// LUCENE-7570
|
||||||
|
public void testDeadlockStalledMerges() throws Exception {
|
||||||
|
Directory dir = newDirectory();
|
||||||
|
IndexWriterConfig iwc = new IndexWriterConfig();
|
||||||
|
|
||||||
|
// so we merge every 2 segments:
|
||||||
|
LogMergePolicy mp = new LogDocMergePolicy();
|
||||||
|
mp.setMergeFactor(2);
|
||||||
|
iwc.setMergePolicy(mp);
|
||||||
|
CountDownLatch done = new CountDownLatch(1);
|
||||||
|
ConcurrentMergeScheduler cms = new ConcurrentMergeScheduler() {
|
||||||
|
@Override
|
||||||
|
protected void doMerge(IndexWriter writer, MergePolicy.OneMerge merge) throws IOException {
|
||||||
|
// let merge takes forever, until commit thread is stalled
|
||||||
|
try {
|
||||||
|
done.await();
|
||||||
|
} catch (InterruptedException ie) {
|
||||||
|
Thread.currentThread().interrupt();
|
||||||
|
throw new RuntimeException(ie);
|
||||||
|
}
|
||||||
|
super.doMerge(writer, merge);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected synchronized void doStall() {
|
||||||
|
done.countDown();
|
||||||
|
super.doStall();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void handleMergeException(Directory dir, Throwable exc) {
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
// so we stall once the 2nd merge wants to run:
|
||||||
|
cms.setMaxMergesAndThreads(1, 1);
|
||||||
|
iwc.setMergeScheduler(cms);
|
||||||
|
|
||||||
|
// so we write a segment every 2 indexed docs:
|
||||||
|
iwc.setMaxBufferedDocs(2);
|
||||||
|
|
||||||
|
final IndexWriter w = new IndexWriter(dir, iwc) {
|
||||||
|
@Override
|
||||||
|
void mergeSuccess(MergePolicy.OneMerge merge) {
|
||||||
|
// tragedy strikes!
|
||||||
|
throw new OutOfMemoryError();
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
w.addDocument(new Document());
|
||||||
|
w.addDocument(new Document());
|
||||||
|
// w writes first segment
|
||||||
|
w.addDocument(new Document());
|
||||||
|
w.addDocument(new Document());
|
||||||
|
// w writes second segment, and kicks off merge, that takes forever (done.await)
|
||||||
|
w.addDocument(new Document());
|
||||||
|
w.addDocument(new Document());
|
||||||
|
// w writes third segment
|
||||||
|
w.addDocument(new Document());
|
||||||
|
w.commit();
|
||||||
|
// w writes fourth segment, and commit flushes and kicks off merge that stalls
|
||||||
|
w.close();
|
||||||
|
dir.close();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue