mirror of https://github.com/apache/lucene.git
LUCENE-5374: Call IW#processEvents before IndexWriter is closed
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1552710 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
5f17bb8a01
commit
ad6c85ef96
|
@ -104,6 +104,12 @@ Bug fixes
|
|||
* LUCENE-5285: Improved highlighting of multi-valued fields with
|
||||
FastVectorHighlighter. (Nik Everett via Adrien Grand)
|
||||
|
||||
* LUCENE-5374: IndexWriter processes internal events after the it
|
||||
closed itself internally. This rare condition can happen if an
|
||||
IndexWriter has internal changes that were not fully applied yet
|
||||
like when index / flush requests happen concurrently to the close or
|
||||
rollback call. (Simon Willnauer)
|
||||
|
||||
Changes in Runtime Behavior
|
||||
|
||||
* LUCENE-5362: IndexReader and SegmentCoreReaders now throw
|
||||
|
|
|
@ -1028,7 +1028,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
|
|||
if (doFlush) {
|
||||
commitInternal();
|
||||
}
|
||||
|
||||
processEvents(false, true);
|
||||
synchronized(this) {
|
||||
// commitInternal calls ReaderPool.commit, which
|
||||
// writes any pending liveDocs from ReaderPool, so
|
||||
|
@ -1063,7 +1063,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
|
|||
}
|
||||
// finally, restore interrupt status:
|
||||
if (interrupted) Thread.currentThread().interrupt();
|
||||
processEvents(false, true);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue