mirror of https://github.com/apache/lucene.git
LUCENE-9309: ensure stopMerges is set under IW lock
This commit is contained in:
parent
2602269f3e
commit
8c1f9815db
|
@ -2282,8 +2282,12 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
|
|||
}
|
||||
|
||||
try {
|
||||
stopMerges = true; // this disables merges forever
|
||||
abortMerges();
|
||||
synchronized (this) {
|
||||
// must be synced otherwise register merge might throw and exception if stopMerges
|
||||
// changes concurrently, abortMerges is synced as well
|
||||
stopMerges = true; // this disables merges forever
|
||||
abortMerges();
|
||||
}
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "rollback: done finish merges");
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue