You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by si...@apache.org on 2020/04/10 11:09:33 UTC

[lucene-solr] 02/02: LUCENE-9309: Wait for #addIndexes merges when aborting merges (#1418)

This is an automated email from the ASF dual-hosted git repository.

simonw pushed a commit to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git

commit 29c7f07de2b5ccb65840ed11987d4b7750f1f070
Author: Simon Willnauer <si...@apache.org>
AuthorDate: Fri Apr 10 12:55:02 2020 +0200

    LUCENE-9309: Wait for #addIndexes merges when aborting merges (#1418)
    
    The SegmentMerger usage in IW#addIndexes(CodecReader...) might make changes
    to the Directory while the IW tries to clean-up files on rollback. This
    causes issues like FileNotFoundExceptions when IDF tries to remove temp files.
    This changes adds a waiting mechanism to the abortMerges method that, in addition
    to the running merges, also waits for merges in addIndices(CodecReader...)
---
 lucene/CHANGES.txt                                 |  2 ++
 .../java/org/apache/lucene/index/IndexWriter.java  | 32 +++++++++++++---------
 .../org/apache/lucene/index/SegmentMerger.java     |  2 +-
 3 files changed, 22 insertions(+), 14 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 3c1987a..1875ca0 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -57,6 +57,8 @@ Bug Fixes
 * LUCENE-9300: Fix corruption of the new gen field infos when doc values updates are applied on a segment created
   externally and added to the index with IndexWriter#addIndexes(Directory). (Jim Ferenczi, Adrien Grand)
 
+* LUCENE-9309: Wait for #addIndexes merges when aborting merges. (Simon Willnauer)
+
 Other
 ---------------------
 
diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
index 23637a4..d9cf673 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
@@ -371,10 +371,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
         }
       }
     }
-
-    int availablePermits() {
-      return permits.availablePermits();
-    }
   }
 
   final IndexFileDeleter deleter;
@@ -397,11 +393,12 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
   HashSet<SegmentCommitInfo> mergingSegments = new HashSet<>();
 
   private final MergeScheduler mergeScheduler;
+  private Set<SegmentMerger> runningAddIndexesMerges = new HashSet<>();
   private LinkedList<MergePolicy.OneMerge> pendingMerges = new LinkedList<>();
   private Set<MergePolicy.OneMerge> runningMerges = new HashSet<>();
   private List<MergePolicy.OneMerge> mergeExceptions = new ArrayList<>();
   private long mergeGen;
-  private boolean stopMerges;
+  private boolean stopMerges; // TODO make sure this is only changed once and never set back to false
   private boolean didMessageState;
 
   final AtomicInteger flushCount = new AtomicInteger();
@@ -2302,6 +2299,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
     }
     
     try {
+      stopMerges = true; // this disables merges forever
       abortMerges();
       if (infoStream.isEnabled("IW")) {
         infoStream.message("IW", "rollback: done finish merges");
@@ -2464,8 +2462,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
             try {
               // Abort any running merges
               abortMerges();
-              // Let merges run again
-              stopMerges = false;
               adjustPendingNumDocs(-segmentInfos.totalMaxDoc());
               // Remove all segments
               segmentInfos.clear();
@@ -2508,9 +2504,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
    *  method: when you abort a long-running merge, you lose
    *  a lot of work that must later be redone. */
   private synchronized void abortMerges() {
-
-    stopMerges = true;
-
     // Abort all pending & running merges:
     for (final MergePolicy.OneMerge merge : pendingMerges) {
       if (infoStream.isEnabled("IW")) {
@@ -2531,10 +2524,11 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
     // We wait here to make all merges stop.  It should not
     // take very long because they periodically check if
     // they are aborted.
-    while (runningMerges.size() != 0) {
+    while (runningMerges.size() + runningAddIndexesMerges.size() != 0) {
 
       if (infoStream.isEnabled("IW")) {
-        infoStream.message("IW", "now wait for " + runningMerges.size() + " running merge/s to abort");
+        infoStream.message("IW", "now wait for " + runningMerges.size()
+            + " running merge/s to abort; currently running addIndexes: " + runningAddIndexesMerges.size());
       }
 
       doWait();
@@ -3010,7 +3004,19 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
         return docWriter.deleteQueue.getNextSequenceNumber();
       }
 
-      merger.merge();                // merge 'em
+      synchronized (this) {
+        ensureOpen();
+        assert stopMerges == false;
+        runningAddIndexesMerges.add(merger);
+      }
+      try {
+        merger.merge();  // merge 'em
+      } finally {
+        synchronized (this) {
+          runningAddIndexesMerges.remove(merger);
+          notifyAll();
+        }
+      }
       SegmentCommitInfo infoPerCommit = new SegmentCommitInfo(info, 0, numSoftDeleted, -1L, -1L, -1L);
 
       info.setFiles(new HashSet<>(trackingDir.getCreatedFiles()));
diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java b/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java
index a078020..4f55497 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java
@@ -208,7 +208,7 @@ final class SegmentMerger {
     }
   }
   
-  public void mergeFieldInfos() throws IOException {
+  public void mergeFieldInfos() {
     for (FieldInfos readerFieldInfos : mergeState.fieldInfos) {
       for (FieldInfo fi : readerFieldInfos) {
         fieldInfosBuilder.add(fi);