Index: lucene/core/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java (revision 1364613) +++ lucene/core/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java (working copy) @@ -243,7 +243,7 @@ sync(); } - /** Wait for any running merge threads to finish */ + /** Wait for any running merge threads to finish. This call is not interruptible as used by {@link #close()}. */ public void sync() { while (true) { MergeThread toSync = null; @@ -259,7 +259,7 @@ try { toSync.join(); } catch (InterruptedException ie) { - throw new ThreadInterruptedException(ie); + // ignore this Exception, we will retry until all threads are dead } } else { break; Index: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (revision 1364613) +++ lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (working copy) @@ -842,7 +842,7 @@ if (hitOOM) { rollbackInternal(); } else { - closeInternal(waitForMerges, !hitOOM); + closeInternal(waitForMerges, true); } } } @@ -891,19 +891,41 @@ docWriter.abort(); // already closed } - if (waitForMerges) - // Give merge scheduler last chance to run, in case - // any pending merges are waiting: - mergeScheduler.merge(this); - + ThreadInterruptedException priorE = null; + if (waitForMerges) { + try { + // Give merge scheduler last chance to run, in case + // any pending merges are waiting: + mergeScheduler.merge(this); + } catch (ThreadInterruptedException tie) { + priorE = tie; + } + } + mergePolicy.close(); + boolean success = false; synchronized(this) { - finishMerges(waitForMerges); + while (!success) { + try { + finishMerges(waitForMerges); + success = true; + } catch (ThreadInterruptedException tie) { + // change to waitForMerges = false, so we just try + // to let the scheduler stop: + waitForMerges = false; + if (priorE == null) priorE = tie; + } + } stopMerges = true; } + + // shutdown scheduler and all threads (this call is not interruptible): mergeScheduler.close(); - + + // if any interrupt occurred we now throw the prior Exception: + if (priorE != null) throw priorE; + if (infoStream.isEnabled("IW")) { infoStream.message("IW", "now call final commit()"); } Index: lucene/core/src/java/org/apache/lucene/index/MergeScheduler.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/MergeScheduler.java (revision 1364613) +++ lucene/core/src/java/org/apache/lucene/index/MergeScheduler.java (working copy) @@ -17,6 +17,7 @@ * limitations under the License. */ +import java.io.Closeable; import java.io.IOException; /**

Expert: {@link IndexWriter} uses an instance @@ -26,7 +27,7 @@ * * @lucene.experimental */ -public abstract class MergeScheduler { +public abstract class MergeScheduler implements Closeable { /** Run the merges provided by {@link IndexWriter#getNextMerge()}. */ public abstract void merge(IndexWriter writer) throws IOException;