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/06/24 19:29:24 UTC

[lucene-solr] branch master updated: LUCENE-9408: Ensure OneMerge#mergeFinished is only called once (#1590)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new f47de19  LUCENE-9408: Ensure OneMerge#mergeFinished is only called once (#1590)
f47de19 is described below

commit f47de19c4e38a6886593899d22ecb18665652e77
Author: Simon Willnauer <si...@apache.org>
AuthorDate: Wed Jun 24 21:28:49 2020 +0200

    LUCENE-9408: Ensure OneMerge#mergeFinished is only called once (#1590)
    
    in the case of an exception it's possible that some OneMerge instances
    will be closed multiple times. This commit ensures that mergeFinished is
    really just called once instead of multiple times.
---
 .../java/org/apache/lucene/index/IndexWriter.java  | 44 ++++++++++++----------
 .../java/org/apache/lucene/index/MergePolicy.java  | 10 ++---
 .../org/apache/lucene/index/TestMergePolicy.java   |  1 -
 3 files changed, 29 insertions(+), 26 deletions(-)

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 eefe04a..fd17220 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
@@ -29,6 +29,7 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
+import java.util.Objects;
 import java.util.Queue;
 import java.util.Set;
 import java.util.concurrent.ConcurrentLinkedQueue;
@@ -4286,25 +4287,30 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
 
   @SuppressWarnings("try")
   private synchronized void closeMergeReaders(MergePolicy.OneMerge merge, boolean suppressExceptions) throws IOException {
-    final boolean drop = suppressExceptions == false;
-    try (Closeable finalizer = () -> merge.mergeFinished(suppressExceptions == false)) {
-      IOUtils.applyToAll(merge.readers, sr -> {
-        final ReadersAndUpdates rld = getPooledInstance(sr.getOriginalSegmentInfo(), false);
-        // We still hold a ref so it should not have been removed:
-        assert rld != null;
-        if (drop) {
-          rld.dropChanges();
-        } else {
-          rld.dropMergingUpdates();
-        }
-        rld.release(sr);
-        release(rld);
-        if (drop) {
-          readerPool.drop(rld.info);
-        }
-      });
-    } finally {
-      Collections.fill(merge.readers, null);
+    if (merge.hasFinished() == false) {
+      final boolean drop = suppressExceptions == false;
+      try (Closeable finalizer = () -> merge.mergeFinished(suppressExceptions == false)) {
+        IOUtils.applyToAll(merge.readers, sr -> {
+          final ReadersAndUpdates rld = getPooledInstance(sr.getOriginalSegmentInfo(), false);
+          // We still hold a ref so it should not have been removed:
+          assert rld != null;
+          if (drop) {
+            rld.dropChanges();
+          } else {
+            rld.dropMergingUpdates();
+          }
+          rld.release(sr);
+          release(rld);
+          if (drop) {
+            readerPool.drop(rld.info);
+          }
+        });
+      } finally {
+        Collections.fill(merge.readers, null);
+      }
+    } else {
+      assert merge.readers.stream().filter(Objects::nonNull).count() == 0 : "we are done but still have readers: " + merge.readers;
+      assert suppressExceptions : "can't be done and not suppressing exceptions";
     }
   }
 
diff --git a/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java b/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java
index ed36c70..91d2302 100644
--- a/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java
+++ b/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java
@@ -256,11 +256,9 @@ public abstract class MergePolicy {
     /** Called by {@link IndexWriter} after the merge is done and all readers have been closed.
      * @param success true iff the merge finished successfully ie. was committed */
     public void mergeFinished(boolean success) throws IOException {
-      mergeCompleted.complete(success);
-      // https://issues.apache.org/jira/browse/LUCENE-9408
-      // if (mergeCompleted.complete(success) == false) {
-      //   throw new IllegalStateException("merge has already finished");
-      // }
+      if (mergeCompleted.complete(success) == false) {
+        throw new IllegalStateException("merge has already finished");
+      }
     }
 
     /** Wrap the reader in order to add/remove information to the merged segment. */
@@ -390,7 +388,7 @@ public abstract class MergePolicy {
      * Returns true if the merge has finished or false if it's still running or
      * has not been started. This method will not block.
      */
-    boolean isDone() {
+    boolean hasFinished() {
       return mergeCompleted.isDone();
     }
 
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestMergePolicy.java b/lucene/core/src/test/org/apache/lucene/index/TestMergePolicy.java
index e5f5635..c252da0 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestMergePolicy.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestMergePolicy.java
@@ -110,7 +110,6 @@ public class TestMergePolicy extends LuceneTestCase {
     }
   }
 
-  @AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/LUCENE-9408")
   public void testFinishTwice() throws IOException {
     try (Directory dir = newDirectory()) {
       MergePolicy.MergeSpecification spec = createRandomMergeSpecification(dir, 1);