LUCENE-6094: allow IW.rollback to stop CMS's stalling too

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1643508 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael McCandless 2014-12-06 09:15:39 +00:00
parent 66745d3ce7
commit df4a217d15
4 changed files with 80 additions and 5 deletions

View File

@ -358,6 +358,9 @@ Bug Fixes
* LUCENE-5987: IndexWriter will now forcefully close itself on
aborting exception (an exception that would otherwise cause silent
data loss). (Robert Muir, Mike McCandless)
* LUCENE-6094: Allow IW.rollback to stop ConcurrentMergeScheduler even
when it's stalling because there are too many merges. (Mike McCandless)
Documentation

View File

@ -300,8 +300,11 @@ public class ConcurrentMergeScheduler extends MergeScheduler {
protected synchronized int mergeThreadCount() {
int count = 0;
for (MergeThread mt : mergeThreads) {
if (mt.isAlive() && mt.getCurrentMerge() != null) {
count++;
if (mt.isAlive()) {
MergePolicy.OneMerge merge = mt.getCurrentMerge();
if (merge != null && merge.isAborted() == false) {
count++;
}
}
}
return count;
@ -399,7 +402,8 @@ public class ConcurrentMergeScheduler extends MergeScheduler {
message(" too many merges; stalling...");
}
try {
wait();
// Only wait 0.25 seconds, so if all merges are aborted (by IW.rollback) we notice:
wait(250);
} catch (InterruptedException ie) {
throw new ThreadInterruptedException(ie);
}

View File

@ -209,8 +209,8 @@ public abstract class MergePolicy {
while (paused) {
try {
// In theory we could wait() indefinitely, but we
// do 1000 msec, defensively
wait(1000);
// do 250 msec, defensively
wait(250);
} catch (InterruptedException ie) {
throw new RuntimeException(ie);
}

View File

@ -482,4 +482,72 @@ public class TestConcurrentMergeScheduler extends LuceneTestCase {
w.close();
dir.close();
}
// LUCENE-6094
public void testHangDuringRollback() throws Throwable {
Directory dir = newMockDirectory();
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
iwc.setMaxBufferedDocs(2);
LogDocMergePolicy mp = new LogDocMergePolicy();
iwc.setMergePolicy(mp);
mp.setMergeFactor(2);
final CountDownLatch mergeStart = new CountDownLatch(1);
final CountDownLatch mergeFinish = new CountDownLatch(1);
ConcurrentMergeScheduler cms = new ConcurrentMergeScheduler() {
@Override
protected void doMerge(MergePolicy.OneMerge merge) throws IOException {
mergeStart.countDown();
try {
mergeFinish.await();
} catch (InterruptedException ie) {
throw new RuntimeException(ie);
}
super.doMerge(merge);
}
};
cms.setMaxMergesAndThreads(1, 1);
iwc.setMergeScheduler(cms);
final IndexWriter w = new IndexWriter(dir, iwc);
w.addDocument(new Document());
w.addDocument(new Document());
// flush
w.addDocument(new Document());
w.addDocument(new Document());
// flush + merge
// Wait for merge to kick off
mergeStart.await();
new Thread() {
@Override
public void run() {
try {
w.addDocument(new Document());
w.addDocument(new Document());
// flush
w.addDocument(new Document());
// W/o the fix for LUCENE-6094 we would hang forever here:
w.addDocument(new Document());
// flush + merge
// Now allow first merge to finish:
mergeFinish.countDown();
} catch (Exception e) {
throw new RuntimeException(e);
}
}
}.start();
while (w.numDocs() != 8) {
Thread.sleep(10);
}
w.rollback();
dir.close();
}
}