You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by si...@apache.org on 2011/08/10 09:28:44 UTC

svn commit: r1156052 - in /lucene/dev/branches/branch_3x: ./ lucene/ lucene/backwards/ lucene/backwards/src/test-framework/ lucene/backwards/src/test/ lucene/backwards/src/test/org/apache/lucene/index/ lucene/src/java/org/apache/lucene/index/ lucene/sr...

Author: simonw
Date: Wed Aug 10 07:28:43 2011
New Revision: 1156052

URL: http://svn.apache.org/viewvc?rev=1156052&view=rev
Log:
LUCENE-3368: IndexWriter commits update documents without corresponding delete

Added:
    lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/index/TestStressNRT.java
      - copied, changed from r1155278, lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestStressNRT.java
Modified:
    lucene/dev/branches/branch_3x/   (props changed)
    lucene/dev/branches/branch_3x/lucene/   (props changed)
    lucene/dev/branches/branch_3x/lucene/CHANGES.txt
    lucene/dev/branches/branch_3x/lucene/backwards/   (props changed)
    lucene/dev/branches/branch_3x/lucene/backwards/src/test/   (props changed)
    lucene/dev/branches/branch_3x/lucene/backwards/src/test-framework/   (props changed)
    lucene/dev/branches/branch_3x/lucene/backwards/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/IndexWriter.java
    lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java
    lucene/dev/branches/branch_3x/solr/   (props changed)

Modified: lucene/dev/branches/branch_3x/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/CHANGES.txt?rev=1156052&r1=1156051&r2=1156052&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/branch_3x/lucene/CHANGES.txt Wed Aug 10 07:28:43 2011
@@ -38,6 +38,9 @@ Bug fixes
   this is fixed if you supply Version >= 3.4 If you supply a previous 
   lucene version, you get the old buggy behavior for backwards compatibility.  
   (Trejkaz, Robert Muir)
+  
+* LUCENE-3368: IndexWriter commits segments without applying their buffered
+  deletes when flushing concurrently. (Simon Willnauer, Mike McCandless)
 
 New Features
 

Modified: lucene/dev/branches/branch_3x/lucene/backwards/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/backwards/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java?rev=1156052&r1=1156051&r2=1156052&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/backwards/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java (original)
+++ lucene/dev/branches/branch_3x/lucene/backwards/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java Wed Aug 10 07:28:43 2011
@@ -486,7 +486,7 @@ public class TestIndexWriterOnDiskFull e
       fail("fake disk full IOExceptions not hit");
     } catch (IOException ioe) {
       // expected
-      assertTrue(ftdm.didFail1);
+      assertTrue(ftdm.didFail1 || ftdm.didFail2);
     }
     _TestUtil.checkIndex(dir);
     ftdm.clearDoFail();

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=1156052&r1=1156051&r2=1156052&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 10 07:28:43 2011
@@ -3360,21 +3360,54 @@ public class IndexWriter implements Clos
    *  only "stick" if there are actually changes in the
    *  index to commit.
    */
-  public final void prepareCommit(Map<String,String> commitUserData) throws CorruptIndexException, IOException {
+  public final void prepareCommit(Map<String, String> commitUserData)
+      throws CorruptIndexException, IOException {
 
     if (hitOOM) {
-      throw new IllegalStateException("this writer hit an OutOfMemoryError; cannot commit");
+      throw new IllegalStateException(
+          "this writer hit an OutOfMemoryError; cannot commit");
     }
 
     if (pendingCommit != null)
-      throw new IllegalStateException("prepareCommit was already called with no corresponding call to commit");
+      throw new IllegalStateException(
+          "prepareCommit was already called with no corresponding call to commit");
 
     if (infoStream != null)
       message("prepareCommit: flush");
 
-    flush(true, true);
+    ensureOpen(false);
+    final boolean anySegmentsFlushed;
+    final SegmentInfos toCommit;
+    synchronized (this) {
+      anySegmentsFlushed = doFlush(true);
+      readerPool.commit(segmentInfos);
+      // Must clone the segmentInfos while we still
+      // hold fullFlushLock and while sync'd so that
+      // no partial changes (eg a delete w/o
+      // corresponding add from an updateDocument) can
+      // sneak into the commit point:
+      toCommit = (SegmentInfos) segmentInfos.clone();
+      pendingCommitChangeCount = changeCount;
+      // 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(toCommit, false);
+    }
+    boolean success = false;
+    try {
+      if (anySegmentsFlushed) {
+        maybeMerge();
+      } 
+      success = true;
+    } finally {
+      if (!success) {
+        deleter.decRef(toCommit);
+      }
+    }
 
-    startCommit(commitUserData);
+    startCommit(toCommit, commitUserData);
   }
 
   // Used only by commit, below; lock order is commitLock -> IW
@@ -3602,6 +3635,7 @@ public class IndexWriter implements Clos
       } else if (infoStream != null) {
         message("don't apply deletes now delTermCount=" + bufferedDeletesStream.numTerms() + " bytesUsed=" + bufferedDeletesStream.bytesUsed());
       }
+      
 
       doAfterFlush();
       flushCount.incrementAndGet();
@@ -4460,7 +4494,7 @@ public class IndexWriter implements Clos
    *  if it wasn't already.  If that succeeds, then we
    *  prepare a new segments_N file but do not fully commit
    *  it. */
-  private void startCommit(Map<String,String> commitUserData) throws IOException {
+  private void startCommit(SegmentInfos toSync, Map<String,String> commitUserData) throws IOException {
 
     assert testPoint("startStartCommit");
     assert pendingCommit == null;
@@ -4474,17 +4508,16 @@ public class IndexWriter implements Clos
       if (infoStream != null)
         message("startCommit(): start");
 
-      final SegmentInfos toSync;
-      final long myChangeCount;
 
       synchronized(this) {
 
         assert lastCommitChangeCount <= changeCount;
-        myChangeCount = changeCount;
         
-        if (changeCount == lastCommitChangeCount) {
-          if (infoStream != null)
+        if (pendingCommitChangeCount == lastCommitChangeCount) {
+          if (infoStream != null) {
             message("  skip startCommit(): no changes pending");
+          }
+          deleter.decRef(toSync);
           return;
         }
         
@@ -4493,22 +4526,13 @@ public class IndexWriter implements Clos
         // referenced by toSync, in the background.
         
         if (infoStream != null)
-          message("startCommit index=" + segString(segmentInfos) + " changeCount=" + changeCount);
-
-        readerPool.commit(segmentInfos);
-        toSync = (SegmentInfos) segmentInfos.clone();
+          message("startCommit index=" + segString(toSync) + " changeCount=" + changeCount);
 
         assert filesExist(toSync);
         
-        if (commitUserData != null)
+        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);
+        }
       }
 
       assert testPoint("midStartCommit");
@@ -4532,14 +4556,13 @@ public class IndexWriter implements Clos
           // (this method unwinds everything it did on
           // an exception)
           toSync.prepareCommit(directory);
-
-          pendingCommit = toSync;
           pendingCommitSet = true;
-          pendingCommitChangeCount = myChangeCount;
+          pendingCommit = toSync;
         }
 
-        if (infoStream != null)
+        if (infoStream != null) {
           message("done all syncs");
+        }
 
         assert testPoint("midStartCommitSuccess");
 

Modified: lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java?rev=1156052&r1=1156051&r2=1156052&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java (original)
+++ lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java Wed Aug 10 07:28:43 2011
@@ -486,7 +486,7 @@ public class TestIndexWriterOnDiskFull e
       fail("fake disk full IOExceptions not hit");
     } catch (IOException ioe) {
       // expected
-      assertTrue(ftdm.didFail1);
+      assertTrue(ftdm.didFail1 || ftdm.didFail2);
     }
     _TestUtil.checkIndex(dir);
     ftdm.clearDoFail();

Copied: lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/index/TestStressNRT.java (from r1155278, lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestStressNRT.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/index/TestStressNRT.java?p2=lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/index/TestStressNRT.java&p1=lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestStressNRT.java&r1=1155278&r2=1156052&rev=1156052&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestStressNRT.java (original)
+++ lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/index/TestStressNRT.java Wed Aug 10 07:28:43 2011
@@ -74,6 +74,7 @@ public class TestStressNRT extends Lucen
     final int maxConcurrentCommits = _TestUtil.nextInt(random, 1, TEST_NIGHTLY ? 10 : 5);   // number of committers at a time... needed if we want to avoid commit errors due to exceeding the max
     
     final boolean tombstones = random.nextBoolean();
+    
 
     // query variables
     final AtomicLong operations = new AtomicLong(atLeast(50000));  // number of query operations to perform in total