mirror of https://github.com/apache/lucene.git
Fix waitForMerges to allow merge schedule to run one last time
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1618667 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
235a886169
commit
52b2a30bcc
|
@ -908,7 +908,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
infoStream.message("IW", "now flush at close");
|
||||
}
|
||||
flush(true, true);
|
||||
finishMerges(true);
|
||||
waitForMerges();
|
||||
commitInternal(config.getMergePolicy());
|
||||
rollbackInternal(); // ie close, since we just committed
|
||||
success = true;
|
||||
|
@ -1999,7 +1999,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
|
||||
try {
|
||||
synchronized(this) {
|
||||
finishMerges(false);
|
||||
abortMerges();
|
||||
stopMerges = true;
|
||||
}
|
||||
|
||||
|
@ -2136,7 +2136,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
synchronized (this) {
|
||||
try {
|
||||
// Abort any running merges
|
||||
finishMerges(false);
|
||||
abortMerges();
|
||||
// Remove all segments
|
||||
segmentInfos.clear();
|
||||
// Ask deleter to locate unreferenced files & remove them:
|
||||
|
@ -2173,13 +2173,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
/** Aborts running merges. Be careful when using this
|
||||
* method: when you abort a long-running merge, you lose
|
||||
* a lot of work that must later be redone. */
|
||||
public void abortMerges() {
|
||||
finishMerges(false);
|
||||
}
|
||||
|
||||
private synchronized void finishMerges(boolean waitForMerges) {
|
||||
if (!waitForMerges) {
|
||||
|
||||
public synchronized void abortMerges() {
|
||||
stopMerges = true;
|
||||
|
||||
// Abort all pending & running merges:
|
||||
|
@ -2219,15 +2213,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "all running merges have aborted");
|
||||
}
|
||||
|
||||
} else {
|
||||
// waitForMerges() will ensure any running addIndexes finishes.
|
||||
// It's fine if a new one attempts to start because from our
|
||||
// caller above the call will see that we are in the
|
||||
// process of closing, and will throw an
|
||||
// AlreadyClosedException.
|
||||
waitForMerges();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -2236,11 +2221,20 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
* <p>It is guaranteed that any merges started prior to calling this method
|
||||
* will have completed once this method completes.</p>
|
||||
*/
|
||||
public synchronized void waitForMerges() {
|
||||
public void waitForMerges() throws IOException {
|
||||
|
||||
// Give merge scheduler last chance to run, in case
|
||||
// any pending merges are waiting. We can't hold IW's lock
|
||||
// when going into merge because it can lead to deadlock.
|
||||
mergeScheduler.merge(this, MergeTrigger.CLOSING, false);
|
||||
|
||||
synchronized (this) {
|
||||
ensureOpen(false);
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "waitForMerges");
|
||||
}
|
||||
|
||||
|
||||
while (pendingMerges.size() > 0 || runningMerges.size() > 0) {
|
||||
doWait();
|
||||
}
|
||||
|
@ -2252,6 +2246,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
infoStream.message("IW", "waitForMerges done");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Called whenever the SegmentInfos has been updated and
|
||||
|
@ -3835,7 +3830,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
* the synchronized lock on IndexWriter instance. */
|
||||
final synchronized void mergeFinish(MergePolicy.OneMerge merge) {
|
||||
|
||||
// forceMerge, addIndexes or finishMerges may be waiting
|
||||
// forceMerge, addIndexes or waitForMerges may be waiting
|
||||
// on merges to finish.
|
||||
notifyAll();
|
||||
|
||||
|
|
|
@ -233,7 +233,7 @@ public class TestIndexWriterMergePolicy extends LuceneTestCase {
|
|||
writer.addDocument(doc);
|
||||
}
|
||||
|
||||
private void checkInvariants(IndexWriter writer) {
|
||||
private void checkInvariants(IndexWriter writer) throws IOException {
|
||||
writer.waitForMerges();
|
||||
int maxBufferedDocs = writer.getConfig().getMaxBufferedDocs();
|
||||
int mergeFactor = ((LogMergePolicy) writer.getConfig().getMergePolicy()).getMergeFactor();
|
||||
|
|
Loading…
Reference in New Issue