You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by us...@apache.org on 2012/07/24 08:07:15 UTC

svn commit: r1364904 - in /lucene/dev/branches/branch_4x: ./ lucene/ lucene/core/ lucene/core/src/java/org/apache/lucene/index/IndexWriter.java

Author: uschindler
Date: Tue Jul 24 06:07:14 2012
New Revision: 1364904

URL: http://svn.apache.org/viewvc?rev=1364904&view=rev
Log:
Merged revision(s) 1364903 from lucene/dev/trunk:
LUCENE-4245: Addon: handle failures during flushing by enforcing CMS to stop

Modified:
    lucene/dev/branches/branch_4x/   (props changed)
    lucene/dev/branches/branch_4x/lucene/   (props changed)
    lucene/dev/branches/branch_4x/lucene/core/   (props changed)
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java?rev=1364904&r1=1364903&r2=1364904&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java Tue Jul 24 06:07:14 2012
@@ -885,44 +885,55 @@ public class IndexWriter implements Clos
 
       docWriter.close();
 
-      // Only allow a new merge to be triggered if we are
-      // going to wait for merges:
-      if (doFlush) {
-        flush(waitForMerges, true);
-      } else {
-        docWriter.abort(); // already closed
-      }
-
-      if (waitForMerges) {
-        try {
-          // Give merge scheduler last chance to run, in case
-          // any pending merges are waiting:
-          mergeScheduler.merge(this);
-        } catch (ThreadInterruptedException tie) {
-          // ignore any interruption, does not matter
-          interrupted = true;
+      try {
+        // Only allow a new merge to be triggered if we are
+        // going to wait for merges:
+        if (doFlush) {
+          flush(waitForMerges, true);
+        } else {
+          docWriter.abort(); // already closed
         }
-      }
+        
+      } finally {
+        // clean up merge scheduler in all cases, although flushing may have failed:
       
-      mergePolicy.close();
-
-      synchronized(this) {
-        for (;;) {
+        if (waitForMerges) {
           try {
-            finishMerges(waitForMerges && !interrupted);
-            break;
+            // Give merge scheduler last chance to run, in case
+            // any pending merges are waiting:
+            mergeScheduler.merge(this);
           } catch (ThreadInterruptedException tie) {
-            // by setting the interrupted status, the
-            // next call to finishMerges will pass false,
-            // so it will not wait
+            // ignore any interruption, does not matter
             interrupted = true;
+            if (infoStream.isEnabled("IW")) {
+              infoStream.message("IW", "interrupted while waiting for final merges");
+            }
           }
         }
-        stopMerges = true;
+        
+        mergePolicy.close();
+
+        synchronized(this) {
+          for (;;) {
+            try {
+              finishMerges(waitForMerges && !interrupted);
+              break;
+            } catch (ThreadInterruptedException tie) {
+              // by setting the interrupted status, the
+              // next call to finishMerges will pass false,
+              // so it will not wait
+              interrupted = true;
+              if (infoStream.isEnabled("IW")) {
+                infoStream.message("IW", "interrupted while waiting for merges to finish");
+              }
+            }
+          }
+          stopMerges = true;
+        }
+        
+        // shutdown scheduler and all threads (this call is not interruptible):
+        mergeScheduler.close();
       }
-      
-      // shutdown scheduler and all threads (this call is not interruptible):
-      mergeScheduler.close();
 
       if (infoStream.isEnabled("IW")) {
         infoStream.message("IW", "now call final commit()");