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 2010/08/11 20:25:13 UTC

svn commit: r984514 - /lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/IndexWriter.java

Author: mikemccand
Date: Wed Aug 11 18:25:12 2010
New Revision: 984514

URL: http://svn.apache.org/viewvc?rev=984514&view=rev
Log:
LUCENE-2576: simplify IndexWriter's private startCommit method now that it's single thread'd (may fix this intermittent test failure); also add one missing checkpoint() in addIndexes

Modified:
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/IndexWriter.java

Modified: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/IndexWriter.java?rev=984514&r1=984513&r2=984514&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/IndexWriter.java Wed Aug 11 18:25:12 2010
@@ -2907,6 +2907,7 @@ public class IndexWriter implements Clos
               info.setUseCompoundFile(true);
             }
           } finally {
+            checkpoint();
             deleter.decRef(files);
           }
         }
@@ -3044,7 +3045,7 @@ public class IndexWriter implements Clos
       handleOOM(oom, "addIndexes(Directory...)");
     }
   }
-  
+
   /**
    * A hook for extending classes to execute operations after pending added and
    * deleted documents have been flushed to the Directory but before the change
@@ -3211,6 +3212,7 @@ public class IndexWriter implements Clos
         setRollbackSegmentInfos(pendingCommit);
         deleter.checkpoint(pendingCommit, true);
       } finally {
+        // Matches the incRef done in startCommit:
         deleter.decRef(pendingCommit);
         pendingCommit = null;
         notifyAll();
@@ -4279,6 +4281,21 @@ public class IndexWriter implements Clos
     }
   }
 
+  // called only from assert
+  private boolean filesExist(SegmentInfos toSync) throws IOException {
+    Collection<String> files = toSync.files(directory, false);
+    for(final String fileName: files) {
+      assert directory.fileExists(fileName): "file " + fileName + " does not exist";
+      // If this trips it means we are missing a call to
+      // .checkpoint somewhere, because by the time we
+      // are called, deleter should know about every
+      // file referenced by the current head
+      // segmentInfos:
+      assert deleter.exists(fileName) : "IndexFileDeleter doesn't know about file " + fileName;
+    }
+    return true;
+  }
+
   /** Walk through all files referenced by the current
    *  segmentInfos and ask the Directory to sync each file,
    *  if it wasn't already.  If that succeeds, then we
@@ -4287,9 +4304,7 @@ public class IndexWriter implements Clos
   private void startCommit(long sizeInBytes, Map<String,String> commitUserData) throws IOException {
 
     assert testPoint("startStartCommit");
-
-    // TODO: as of LUCENE-2095, we can simplify this method,
-    // since only 1 thread can be in here at once
+    assert pendingCommit == null;
 
     if (hitOOM) {
       throw new IllegalStateException("this writer hit an OutOfMemoryError; cannot commit");
@@ -4300,7 +4315,7 @@ public class IndexWriter implements Clos
       if (infoStream != null)
         message("startCommit(): start sizeInBytes=" + sizeInBytes);
 
-      SegmentInfos toSync = null;
+      final SegmentInfos toSync;
       final long myChangeCount;
 
       synchronized(this) {
@@ -4315,94 +4330,50 @@ public class IndexWriter implements Clos
         
         // First, we clone & incref the segmentInfos we intend
         // to sync, then, without locking, we sync() each file
-        // referenced by toSync, in the background.  Multiple
-        // threads can be doing this at once, if say a large
-        // merge and a small merge finish at the same time:
+        // referenced by toSync, in the background.
         
         if (infoStream != null)
           message("startCommit index=" + segString(segmentInfos) + " changeCount=" + changeCount);
-        
+
         readerPool.commit();
         
         toSync = (SegmentInfos) segmentInfos.clone();
+        assert filesExist(toSync);
         
         if (commitUserData != null)
           toSync.setUserData(commitUserData);
         
+        // This protects the segmentInfos we are now going
+        // to commit.  This is important in case, eg, while
+        // we are trying to sync all referenced files, a
+        // merge completes which would otherwise have
+        // removed the files we are now syncing.
         deleter.incRef(toSync, false);
         myChangeCount = changeCount;
-        
-        Collection<String> files = toSync.files(directory, false);
-        for(final String fileName: files) {
-          assert directory.fileExists(fileName): "file " + fileName + " does not exist";
-
-          // If this trips it means we are missing a call to
-          // .checkpoint somewhere, because by the time we
-          // are called, deleter should know about every
-          // file referenced by the current head
-          // segmentInfos:
-          assert deleter.exists(fileName);
-        }
       }
 
       assert testPoint("midStartCommit");
 
-      boolean setPending = false;
-
       try {
+        // This call can take a long time -- 10s of seconds
+        // or more.  We do it without sync:
         directory.sync(toSync.files(directory, false));
 
         assert testPoint("midStartCommit2");
 
         synchronized(this) {
-          // If someone saved a newer version of segments file
-          // since I first started syncing my version, I can
-          // safely skip saving myself since I've been
-          // superseded:
-
-          while(true) {
-            if (myChangeCount <= lastCommitChangeCount) {
-              if (infoStream != null) {
-                message("sync superseded by newer infos");
-              }
-              break;
-            } else if (pendingCommit == null) {
-              // My turn to commit
 
-              if (segmentInfos.getGeneration() > toSync.getGeneration())
-                toSync.updateGeneration(segmentInfos);
+          assert pendingCommit == null;
 
-              boolean success = false;
-              try {
+          assert segmentInfos.getGeneration() == toSync.getGeneration();
 
-                // Exception here means nothing is prepared
-                // (this method unwinds everything it did on
-                // an exception)
-                try {
-                  toSync.prepareCommit(directory);
-                } finally {
-                  // Have our master segmentInfos record the
-                  // generations we just prepared.  We do this
-                  // on error or success so we don't
-                  // double-write a segments_N file.
-                  segmentInfos.updateGeneration(toSync);
-                }
-
-                assert pendingCommit == null;
-                setPending = true;
-                pendingCommit = toSync;
-                pendingCommitChangeCount = myChangeCount;
-                success = true;
-              } finally {
-                if (!success && infoStream != null)
-                  message("hit exception committing segments file");
-              }
-              break;
-            } else {
-              // Must wait for other commit to complete
-              doWait();
-            }
-          }
+          // Exception here means nothing is prepared
+          // (this method unwinds everything it did on
+          // an exception)
+          toSync.prepareCommit(directory);
+
+          pendingCommit = toSync;
+          pendingCommitChangeCount = myChangeCount;
         }
 
         if (infoStream != null)
@@ -4412,8 +4383,20 @@ public class IndexWriter implements Clos
 
       } finally {
         synchronized(this) {
-          if (!setPending)
+
+          // Have our master segmentInfos record the
+          // generations we just prepared.  We do this
+          // on error or success so we don't
+          // double-write a segments_N file.
+          segmentInfos.updateGeneration(toSync);
+
+          if (pendingCommit == null) {
+            if (infoStream != null) {
+              message("hit exception committing segments file");
+            }
+
             deleter.decRef(toSync);
+          }
         }
       }
     } catch (OutOfMemoryError oom) {