mirror of https://github.com/apache/lucene.git
LUCENE-4245: use IOUtils to close everything in finally block
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1364931 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
845a3c3d60
commit
2b53e08898
|
@ -893,6 +893,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
}
|
||||
|
||||
} finally {
|
||||
try {
|
||||
// clean up merge scheduler in all cases, although flushing may have failed:
|
||||
interrupted = Thread.interrupted();
|
||||
|
||||
|
@ -910,8 +911,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
}
|
||||
}
|
||||
|
||||
mergePolicy.close();
|
||||
|
||||
synchronized(this) {
|
||||
for (;;) {
|
||||
try {
|
||||
|
@ -930,8 +929,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
stopMerges = true;
|
||||
}
|
||||
|
||||
// shutdown scheduler and all threads (this call is not interruptible):
|
||||
mergeScheduler.close();
|
||||
} finally {
|
||||
// shutdown policy, scheduler and all threads (this call is not interruptible):
|
||||
IOUtils.closeWhileHandlingException(mergePolicy, mergeScheduler);
|
||||
}
|
||||
}
|
||||
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
|
|
Loading…
Reference in New Issue