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