LUCENE-9309: ensure stopMerges is set under IW lock

This commit is contained in:
Simon Willnauer 2020-04-11 19:53:21 +02:00
parent 2602269f3e
commit 8c1f9815db
1 changed files with 6 additions and 2 deletions

View File

@ -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");
}