You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2014/09/23 15:17:32 UTC

svn commit: r1627005 - in /lucene/dev/branches/branch_5x: ./ lucene/ lucene/core/ lucene/core/src/java/org/apache/lucene/index/IndexWriter.java lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java

Author: mikemccand
Date: Tue Sep 23 13:17:31 2014
New Revision: 1627005

URL: http://svn.apache.org/r1627005
Log:
LUCENE-5958: fix IW deadlock

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

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java?rev=1627005&r1=1627004&r2=1627005&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java Tue Sep 23 13:17:31 2014
@@ -2068,15 +2068,15 @@ public class IndexWriter implements Clos
      * get in our way and do unnecessary work. -- if we don't lock this here we might
      * get in trouble if */
     synchronized (fullFlushLock) { 
-        /*
-         * We first abort and trash everything we have in-memory
-         * and keep the thread-states locked, the lockAndAbortAll operation
-         * also guarantees "point in time semantics" ie. the checkpoint that we need in terms
-         * of logical happens-before relationship in the DW. So we do
-         * abort all in memory structures 
-         * We also drop global field numbering before during abort to make
-         * sure it's just like a fresh index.
-         */
+      /*
+       * We first abort and trash everything we have in-memory
+       * and keep the thread-states locked, the lockAndAbortAll operation
+       * also guarantees "point in time semantics" ie. the checkpoint that we need in terms
+       * of logical happens-before relationship in the DW. So we do
+       * abort all in memory structures 
+       * We also drop global field numbering before during abort to make
+       * sure it's just like a fresh index.
+       */
       try {
         docWriter.lockAndAbortAll(this);
         processEvents(false, true);
@@ -2101,8 +2101,6 @@ public class IndexWriter implements Clos
             segmentInfos.changed();
             globalFieldNumberMap.clear();
             success = true;
-          } catch (OutOfMemoryError oom) {
-            tragicEvent(oom, "deleteAll");
           } finally {
             if (!success) {
               if (infoStream.isEnabled("IW")) {
@@ -2111,6 +2109,8 @@ public class IndexWriter implements Clos
             }
           }
         }
+      } catch (OutOfMemoryError oom) {
+        tragicEvent(oom, "deleteAll");
       } finally {
         docWriter.unlockAllAfterAbortAll(this);
       }
@@ -2889,63 +2889,70 @@ public class IndexWriter implements Clos
     }
   }
 
-  private synchronized final void finishCommit() throws IOException {
+  private final void finishCommit() throws IOException {
 
-    boolean success = false;
-    
-    if (pendingCommit != null) {
-      try {
-        if (infoStream.isEnabled("IW")) {
-          infoStream.message("IW", "commit: pendingCommit != null");
-        }
-        pendingCommit.finishCommit(directory);
-        success = true;
-        // we committed, if anything goes wrong after this: we are screwed
-        try {
-          if (infoStream.isEnabled("IW")) {
-            infoStream.message("IW", "commit: wrote segments file \"" + pendingCommit.getSegmentsFileName() + "\"");
-          }
-          segmentInfos.updateGeneration(pendingCommit);
-          lastCommitChangeCount = pendingCommitChangeCount;
-          rollbackSegments = pendingCommit.createBackupSegmentInfos();
-          // NOTE: don't use this.checkpoint() here, because
-          // we do not want to increment changeCount:
-          deleter.checkpoint(pendingCommit, true);
-        } catch (Throwable tragedy) {
-          tragicEvent(tragedy, "finishCommit");
-        }
-      } finally {
-        // Matches the incRef done in prepareCommit:
-        try {
-          if (success == false || tragedy == null) {
+    boolean commitCompleted = false;
+    boolean finished = false;
+    String committedSegmentsFileName = null;
+
+    try {
+      synchronized(this) {
+        if (pendingCommit != null) {
+          try {
+
+            if (infoStream.isEnabled("IW")) {
+              infoStream.message("IW", "commit: pendingCommit != null");
+            }
+
+            committedSegmentsFileName = pendingCommit.finishCommit(directory);
+
+            // we committed, if anything goes wrong after this, we are screwed and it's a tragedy:
+            commitCompleted = true;
+
+            // NOTE: don't use this.checkpoint() here, because
+            // we do not want to increment changeCount:
+            deleter.checkpoint(pendingCommit, true);
+
+            lastCommitChangeCount = pendingCommitChangeCount;
+            rollbackSegments = pendingCommit.createBackupSegmentInfos();
+
+            finished = true;
+          } finally {
+            notifyAll();
             try {
-              deleter.decRef(filesToCommit);
-            } catch (Throwable t) {
-              // if the commit succeeded, we are in screwed state
-              // otherwise, throw our original exception
-              if (success) {
-                tragicEvent(tragedy, "finishCommit");
+              if (finished) {
+                // all is good
+                deleter.decRef(filesToCommit);
+              } else if (commitCompleted == false) {
+                // exc happened in finishCommit: not a tragedy
+                deleter.decRefWhileHandlingException(filesToCommit);
               }
-            } 
+            } finally {
+              pendingCommit = null;
+              filesToCommit = null;
+            }
+          }
+        } else {
+          assert filesToCommit == null;
+          if (infoStream.isEnabled("IW")) {
+            infoStream.message("IW", "commit: pendingCommit == null; skip");
           }
-        } finally {
-          filesToCommit = null;
-          pendingCommit = null;
-          notifyAll();
         }
       }
-
+    } catch (Throwable t) {
       if (infoStream.isEnabled("IW")) {
-        infoStream.message("IW", String.format(Locale.ROOT, "commit: took %.1f msec", (System.nanoTime()-startCommitTime)/1000000.0));
+        infoStream.message("IW", "hit exception during finishCommit: " + t.getMessage());
       }
-      
-    } else {
-      if (infoStream.isEnabled("IW")) {
-        infoStream.message("IW", "commit: pendingCommit == null; skip");
+      if (commitCompleted) {
+        tragicEvent(t, "finishCommit");
+      } else {
+        IOUtils.reThrow(t);
       }
     }
 
     if (infoStream.isEnabled("IW")) {
+      infoStream.message("IW", "commit: wrote segments file \"" + committedSegmentsFileName + "\"");
+      infoStream.message("IW", String.format(Locale.ROOT, "commit: took %.1f msec", (System.nanoTime()-startCommitTime)/1000000.0));
       infoStream.message("IW", "commit: done");
     }
   }
@@ -4375,12 +4382,17 @@ public class IndexWriter implements Clos
   }
 
   private void tragicEvent(Throwable tragedy, String location) {
+    // We cannot hold IW's lock here else it can lead to deadlock:
+    assert Thread.holdsLock(this) == false;
+
     if (infoStream.isEnabled("IW")) {
       infoStream.message("IW", "hit " + tragedy.getClass().getSimpleName() + " inside " + location);
     }
-    // its possible you could have a really bad day
-    if (this.tragedy == null) {
-      this.tragedy = tragedy;
+    synchronized (this) {
+      // its possible you could have a really bad day
+      if (this.tragedy == null) {
+        this.tragedy = tragedy;
+      }
     }
     // if we are already closed (e.g. called by rollback), this will be a no-op.
     synchronized(commitLock) {

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java?rev=1627005&r1=1627004&r2=1627005&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java Tue Sep 23 13:17:31 2014
@@ -704,14 +704,16 @@ public final class SegmentInfos implemen
     return files;
   }
 
-  final void finishCommit(Directory dir) throws IOException {
+  /** Returns the committed segments_N filename. */
+  final String finishCommit(Directory dir) throws IOException {
     if (pendingCommit == false) {
       throw new IllegalStateException("prepareCommit was not called");
     }
     boolean success = false;
+    final String dest;
     try {
       final String src =  IndexFileNames.fileNameFromGeneration(IndexFileNames.PENDING_SEGMENTS, "", generation);
-      final String dest = IndexFileNames.fileNameFromGeneration(IndexFileNames.SEGMENTS,         "", generation);
+      dest = IndexFileNames.fileNameFromGeneration(IndexFileNames.SEGMENTS, "", generation);
       dir.renameFile(src, dest);
       success = true;
     } finally {
@@ -723,6 +725,7 @@ public final class SegmentInfos implemen
 
     pendingCommit = false;
     lastGeneration = generation;
+    return dest;
   }
 
   /** Writes & syncs to the Directory dir, taking care to