You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by dw...@apache.org on 2021/03/10 09:48:21 UTC

[lucene] 03/04: LUCENE-8692: initial attempt at fixing bug

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

dweiss pushed a commit to branch jira/LUCENE-8692
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit 4bd7d6810dbf50ae8f07c1051378f482ff953073
Author: Chris Hostetter <ho...@apache.org>
AuthorDate: Tue Mar 5 16:03:31 2019 -0700

    LUCENE-8692: initial attempt at fixing bug
---
 .../src/java/org/apache/lucene/index/IndexWriter.java | 19 ++++++++++++++++---
 1 file changed, 16 insertions(+), 3 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 f9aaf34..eb3d790 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
@@ -2151,8 +2151,15 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
 
   final void maybeMerge(MergePolicy mergePolicy, MergeTrigger trigger, int maxNumSegments) throws IOException {
     ensureOpen(false);
-    boolean newMergesFound = updatePendingMerges(mergePolicy, trigger, maxNumSegments);
-    mergeScheduler.merge(this, trigger, newMergesFound);
+    try {
+      boolean newMergesFound = updatePendingMerges(mergePolicy, trigger, maxNumSegments);
+      mergeScheduler.merge(this, trigger, newMergesFound);
+    } catch (MergePolicy.MergeAbortedException mae) {
+      throw mae;
+    } catch (Throwable t) {
+      tragicEvent(t, "maybeMerge");
+      throw t;
+    }
   }
 
   private synchronized boolean updatePendingMerges(MergePolicy mergePolicy, MergeTrigger trigger, int maxNumSegments)
@@ -3261,6 +3268,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
           }
         }
       } catch (VirtualMachineError tragedy) {
+        // nocommit: should we be catching Throwable & treating as traggic here?
+        // nocommit: if we get an IOException here does it come direct from dir, or has docWriter already tracked it as traggic?
         tragicEvent(tragedy, "prepareCommit");
         throw tragedy;
       } finally {
@@ -4839,7 +4848,11 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
           segmentInfos.updateGeneration(toSync);
         }
       }
-    } catch (VirtualMachineError tragedy) {
+    } catch (MergePolicy.MergeAbortedException mae) {
+      throw mae;
+    } catch (Throwable tragedy) {
+      // nocommit: simonw has questions/concerns about catching Throwable here
+      // nocommit: can/should we catch something narrower?
       tragicEvent(tragedy, "startCommit");
       throw tragedy;
     }