lucene-java-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From mikemcc...@apache.org
Subject svn commit: r681082 - in /lucene/java/trunk/src: java/org/apache/lucene/index/IndexWriter.java test/org/apache/lucene/index/TestIndexWriter.java
Date Wed, 30 Jul 2008 15:38:17 GMT
Author: mikemccand
Date: Wed Jul 30 08:38:16 2008
New Revision: 681082

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

Modified:
    lucene/java/trunk/src/java/org/apache/lucene/index/IndexWriter.java
    lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexWriter.java

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/IndexWriter.java?rev=681082&r1=681081&r2=681082&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/IndexWriter.java Wed Jul 30 08:38:16
2008
@@ -1618,37 +1618,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)
-      abort();
-
-    synchronized(this) {
-      // Ensure that only one thread actually gets to do the closing:
-      if (!closing) {
-        doClose = true;
-        closing = true;
-      } else
-        doClose = false;
+    if (hitOOM) {
+      rollback();
+      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;
     }
   }
 
@@ -1676,7 +1679,7 @@
 
       if (infoStream != null)
         message("now call final commit()");
-
+      
       commit(0);
 
       if (infoStream != null)
@@ -1702,12 +1705,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");
       }
     }
   }
@@ -2614,28 +2615,18 @@
   public void rollback() throws IOException {
     ensureOpen();
     if (autoCommit)
-      throw new IllegalStateException("abort() can only be called when IndexWriter was opened
with autoCommit=false");
+      throw new IllegalStateException("rollback() can only be called when IndexWriter was
opened with autoCommit=false");
 
-    boolean doClose;
-    synchronized(this) {
-
-      if (pendingCommit != null) {
-        pendingCommit.rollbackCommit(directory);
-        deleter.decRef(pendingCommit);
-        pendingCommit = null;
-        notifyAll();
-      }
+    // Ensure that only one thread actually gets to do the closing:
+    if (shouldClose())
+      rollbackInternal();
+  }
 
-      // Ensure that only one thread actually gets to do the closing:
-      if (!closing) {
-        doClose = true;
-        closing = true;
-      } else
-        doClose = false;
-    }
+  private void rollbackInternal() throws IOException {
 
-    if (doClose) {
+    boolean success = false;
 
+    try {
       finishMerges(false);
 
       // Must pre-close these two, in case they increment
@@ -2645,6 +2636,14 @@
       mergeScheduler.close();
 
       synchronized(this) {
+
+        if (pendingCommit != null) {
+          pendingCommit.rollbackCommit(directory);
+          deleter.decRef(pendingCommit);
+          pendingCommit = null;
+          notifyAll();
+        }
+
         // Keep the same segmentInfos instance but replace all
         // of its SegmentInfo instances.  This is so the next
         // attempt to commit using this instance of IndexWriter
@@ -2655,6 +2654,8 @@
         
         docWriter.abort();
 
+        assert testPoint("rollback before checkpoint");
+
         // Ask deleter to locate unreferenced files & remove
         // them:
         deleter.checkpoint(segmentInfos, false);
@@ -2662,9 +2663,23 @@
       }
 
       lastCommitChangeCount = changeCount;
-      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 during rollback");
+        }
+      }
+    }
+
+    closeInternal(false);
   }
 
   private synchronized void finishMerges(boolean waitForMerges) throws IOException {
@@ -3561,7 +3576,7 @@
 
     assert merge.registerDone;
 
-    // If merge was explicitly aborted, or, if abort() or
+    // If merge was explicitly aborted, or, if rollback() or
     // rollbackTransaction() had been called since our merge
     // started (which results in an unqualified
     // deleter.refresh() call that will remove any index

Modified: lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexWriter.java?rev=681082&r1=681081&r2=681082&view=diff
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexWriter.java (original)
+++ lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexWriter.java Wed Jul 30 08:38:16
2008
@@ -3731,4 +3731,38 @@
     reader.close();
     dir.close();
   }
+
+  // LUCENE-1347
+  public class MockIndexWriter4 extends IndexWriter {
+
+    public MockIndexWriter4(Directory dir, boolean autoCommit, Analyzer a, boolean create,
MaxFieldLength mfl) throws IOException {
+      super(dir, autoCommit, a, create, mfl);
+    }
+
+    boolean doFail;
+
+    boolean testPoint(String name) {
+      if (doFail && name.equals("rollback 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,
IndexWriter.MaxFieldLength.LIMITED);
+
+    addDoc(w);
+    w.doFail = true;
+    try {
+      w.rollback();
+      fail("did not hit intentional RuntimeException");
+    } catch (RuntimeException re) {
+      // expected
+    }
+    
+    w.doFail = false;
+    w.rollback();
+  }
 }



Mime
View raw message