lucene-java-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From mikemcc...@apache.org
Subject svn commit: r681089 - in /lucene/java/branches/lucene_2_3: CHANGES.txt src/java/org/apache/lucene/index/IndexWriter.java src/test/org/apache/lucene/index/TestIndexWriter.java
Date Wed, 30 Jul 2008 15:49:14 GMT
Author: mikemccand
Date: Wed Jul 30 08:49:13 2008
New Revision: 681089

URL: http://svn.apache.org/viewvc?rev=681089&view=rev
Log:
LUCENE-1347: port to 2.3.x: fix hang in rollback() after previous rollback() call hit exception

Modified:
    lucene/java/branches/lucene_2_3/CHANGES.txt
    lucene/java/branches/lucene_2_3/src/java/org/apache/lucene/index/IndexWriter.java
    lucene/java/branches/lucene_2_3/src/test/org/apache/lucene/index/TestIndexWriter.java

Modified: lucene/java/branches/lucene_2_3/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/java/branches/lucene_2_3/CHANGES.txt?rev=681089&r1=681088&r2=681089&view=diff
==============================================================================
--- lucene/java/branches/lucene_2_3/CHANGES.txt (original)
+++ lucene/java/branches/lucene_2_3/CHANGES.txt Wed Jul 30 08:49:13 2008
@@ -45,6 +45,10 @@
 12. LUCENE-1270: Fixed intermittant case where IndexWriter.close()
     would hang after IndexWriter.addIndexesNoOptimize had been
     called.  (Stu Hood via Mike McCandless)
+
+13. LUCENE-1347: Fixed hang in IndexWriter.abort() if it was called
+    again after previously hitting an exception during a prior call.
+    (Mike McCandless)
 	
 Build
 

Modified: lucene/java/branches/lucene_2_3/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/lucene_2_3/src/java/org/apache/lucene/index/IndexWriter.java?rev=681089&r1=681088&r2=681089&view=diff
==============================================================================
--- lucene/java/branches/lucene_2_3/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/java/branches/lucene_2_3/src/java/org/apache/lucene/index/IndexWriter.java Wed
Jul 30 08:49:13 2008
@@ -1166,37 +1166,40 @@
    * then return.
    */
   public void close(boolean waitForMerges) throws CorruptIndexException, IOException {
-    boolean doClose;
 
     // If any methods have hit OutOfMemoryError, then abort
     // on close, in case the internal state of IndexWriter
     // or DocumentsWriter is corrupt
-    if (hitOOM)
+    if (hitOOM) {
       abort();
-
-    synchronized(this) {
-      // Ensure that only one thread actually gets to do the closing:
-      if (!closing) {
-        doClose = true;
-        closing = true;
-      } else
-        doClose = false;
+      return;
     }
-    if (doClose)
+
+    // Ensure that only one thread actually gets to do the closing:
+    if (shouldClose())
       closeInternal(waitForMerges);
-    else
-      // Another thread beat us to it (is actually doing the
-      // close), so we will block until that other thread
-      // has finished closing
-      waitForClose();
   }
 
-  synchronized private void waitForClose() {
-    while(!closed && closing) {
-      try {
-        wait();
-      } catch (InterruptedException ie) {
-      }
+  // Returns true if this thread should attempt to close, or
+  // false if IndexWriter is now closed; else, waits until
+  // another thread finishes closing
+  synchronized private boolean shouldClose() {
+    while(true) {
+      if (!closed) {
+        if (!closing) {
+          closing = true;
+          return true;
+        } else {
+          // Another thread is presently trying to close;
+          // wait until it finishes one way (closes
+          // successfully) or another (fails to close)
+          try {
+            wait();
+          } catch (InterruptedException ie) {
+          }
+        }
+      } else
+        return false;
     }
   }
 
@@ -1265,12 +1268,10 @@
       throw oom;
     } finally {
       synchronized(this) {
-        if (!closed) {
-          closing = false;
-          if (infoStream != null)
-            message("hit exception while closing");
-        }
+        closing = false;
         notifyAll();
+        if (!closed && infoStream != null)
+          message("hit exception while closing");
       }
     }
   }
@@ -2008,17 +2009,16 @@
     if (autoCommit)
       throw new IllegalStateException("abort() can only be called when IndexWriter was opened
with autoCommit=false");
 
-    boolean doClose;
-    synchronized(this) {
-      // Ensure that only one thread actually gets to do the closing:
-      if (!closing) {
-        doClose = true;
-        closing = true;
-      } else
-        doClose = false;
-    }
+    // Ensure that only one thread actually gets to do the closing:
+    if (shouldClose())
+      abortInternal();
+  }
+
+  private void abortInternal() throws IOException {
 
-    if (doClose) {
+    boolean success = false;
+
+    try {
 
       finishMerges(false);
 
@@ -2039,6 +2039,8 @@
 
         docWriter.abort(null);
 
+        assert testPoint("abort before checkpoint");
+
         // Ask deleter to locate unreferenced files & remove
         // them:
         deleter.checkpoint(segmentInfos, false);
@@ -2046,9 +2048,23 @@
       }
 
       commitPending = false;
-      closeInternal(false);
-    } else
-      waitForClose();
+
+      success = true;
+    } catch (OutOfMemoryError oom) {
+      hitOOM = true;
+      throw oom;
+    } finally {
+      synchronized(this) {
+        if (!success) {
+          closing = false;
+          notifyAll();
+          if (infoStream != null)
+            message("hit exception while aborting");
+        }
+      }
+    }
+
+    closeInternal(false);
   }
 
   private synchronized void finishMerges(boolean waitForMerges) throws IOException {
@@ -2205,7 +2221,6 @@
             SegmentInfos sis = new SegmentInfos();	  // read infos from dir
             sis.read(dirs[i]);
             for (int j = 0; j < sis.size(); j++) {
-              final SegmentInfo info = sis.info(j);
               segmentInfos.addElement(sis.info(j));	  // add each info
             }
           }

Modified: lucene/java/branches/lucene_2_3/src/test/org/apache/lucene/index/TestIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/lucene_2_3/src/test/org/apache/lucene/index/TestIndexWriter.java?rev=681089&r1=681088&r2=681089&view=diff
==============================================================================
--- lucene/java/branches/lucene_2_3/src/test/org/apache/lucene/index/TestIndexWriter.java
(original)
+++ lucene/java/branches/lucene_2_3/src/test/org/apache/lucene/index/TestIndexWriter.java
Wed Jul 30 08:49:13 2008
@@ -2311,7 +2311,6 @@
       writer.setMergeFactor(4);
 
       IndexerThread[] threads = new IndexerThread[NUM_THREADS];
-      boolean diskFull = false;
 
       for(int i=0;i<NUM_THREADS;i++)
         threads[i] = new IndexerThread(writer, false);
@@ -2404,7 +2403,6 @@
       dir.setMaxSizeInBytes(4*1024+20*iter);
 
       IndexerThread[] threads = new IndexerThread[NUM_THREADS];
-      boolean diskFull = false;
 
       for(int i=0;i<NUM_THREADS;i++)
         threads[i] = new IndexerThread(writer, true);
@@ -2442,7 +2440,7 @@
   private static class FailOnlyOnAbortOrFlush extends MockRAMDirectory.Failure {
     private boolean onlyOnce;
     public FailOnlyOnAbortOrFlush(boolean onlyOnce) {
-      this.onlyOnce = true;
+      this.onlyOnce = onlyOnce;
     }
     public void eval(MockRAMDirectory dir)  throws IOException {
       if (doFail) {
@@ -2502,7 +2500,6 @@
       writer.setMergeFactor(4);
 
       IndexerThread[] threads = new IndexerThread[NUM_THREADS];
-      boolean diskFull = false;
 
       for(int i=0;i<NUM_THREADS;i++)
         threads[i] = new IndexerThread(writer, true);
@@ -2584,7 +2581,7 @@
   private static class FailOnlyInCloseDocStore extends MockRAMDirectory.Failure {
     private boolean onlyOnce;
     public FailOnlyInCloseDocStore(boolean onlyOnce) {
-      this.onlyOnce = true;
+      this.onlyOnce = onlyOnce;
     }
     public void eval(MockRAMDirectory dir)  throws IOException {
       if (doFail) {
@@ -2624,7 +2621,7 @@
   private static class FailOnlyInWriteSegment extends MockRAMDirectory.Failure {
     private boolean onlyOnce;
     public FailOnlyInWriteSegment(boolean onlyOnce) {
-      this.onlyOnce = true;
+      this.onlyOnce = onlyOnce;
     }
     public void eval(MockRAMDirectory dir)  throws IOException {
       if (doFail) {
@@ -3035,4 +3032,38 @@
     s.close();
     dir.close();
   }
+
+  // LUCENE-1347
+  public class MockIndexWriter4 extends IndexWriter {
+
+    public MockIndexWriter4(Directory dir, boolean autoCommit, Analyzer a, boolean create)
throws IOException {
+      super(dir, autoCommit, a, create);
+    }
+
+    boolean doFail;
+
+    boolean testPoint(String name) {
+      if (doFail && name.equals("abort before checkpoint"))
+        throw new RuntimeException("intentionally failing");
+      return true;
+    }
+  }
+
+  // LUCENE-1347
+  public void testRollbackExceptionHang() throws Throwable {
+    MockRAMDirectory dir = new MockRAMDirectory();
+    MockIndexWriter4 w = new MockIndexWriter4(dir, false, new WhitespaceAnalyzer(), true);
+
+    addDoc(w);
+    w.doFail = true;
+    try {
+      w.abort();
+      fail("did not hit intentional RuntimeException");
+    } catch (RuntimeException re) {
+      // expected
+    }
+    
+    w.doFail = false;
+    w.abort();
+  }
 }



Mime
View raw message