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 2015/04/04 10:36:52 UTC
svn commit: r1671253 - in /lucene/dev/branches/branch_5x/lucene/core/src:
java/org/apache/lucene/index/IndexWriter.java
test/org/apache/lucene/index/TestSnapshotDeletionPolicy.java
Author: mikemccand
Date: Sat Apr 4 08:36:52 2015
New Revision: 1671253
URL: http://svn.apache.org/r1671253
Log:
defensively switch to AtomicLong for IndexWriter's changeCount; make failing test a bit more evil
Modified:
lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestSnapshotDeletionPolicy.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=1671253&r1=1671252&r2=1671253&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 Sat Apr 4 08:36:52 2015
@@ -255,7 +255,7 @@ public class IndexWriter implements Clos
private final Directory mergeDirectory; // used for merging
private final Analyzer analyzer; // how to analyze text
- private volatile long changeCount; // increments every time a change is completed
+ private final AtomicLong changeCount = new AtomicLong(); // increments every time a change is completed
private volatile long lastCommitChangeCount; // last changeCount that was committed
private List<SegmentCommitInfo> rollbackSegments; // list of segmentInfo we will fallback to if the commit fails
@@ -1531,7 +1531,7 @@ public class IndexWriter implements Clos
// could close, re-open and re-return the same segment
// name that was previously returned which can cause
// problems at least with ConcurrentMergeScheduler.
- changeCount++;
+ changeCount.incrementAndGet();
segmentInfos.changed();
return "_" + Integer.toString(segmentInfos.counter++, Character.MAX_RADIX);
}
@@ -1979,7 +1979,7 @@ public class IndexWriter implements Clos
deleter.checkpoint(segmentInfos, false);
deleter.refresh();
- lastCommitChangeCount = changeCount;
+ lastCommitChangeCount = changeCount.get();
deleter.close();
@@ -2094,7 +2094,7 @@ public class IndexWriter implements Clos
// Don't bother saving any changes in our segmentInfos
readerPool.dropAll(false);
// Mark that the index has changed
- ++changeCount;
+ changeCount.incrementAndGet();
segmentInfos.changed();
globalFieldNumberMap.clear();
@@ -2207,13 +2207,13 @@ public class IndexWriter implements Clos
* close/commit we will write a new segments file, but
* does NOT bump segmentInfos.version. */
synchronized void checkpointNoSIS() throws IOException {
- changeCount++;
+ changeCount.incrementAndGet();
deleter.checkpoint(segmentInfos, false);
}
/** Called internally if any index state has changed. */
synchronized void changed() {
- changeCount++;
+ changeCount.incrementAndGet();
segmentInfos.changed();
}
@@ -2717,7 +2717,7 @@ public class IndexWriter implements Clos
// sneak into the commit point:
toCommit = segmentInfos.clone();
- pendingCommitChangeCount = changeCount;
+ pendingCommitChangeCount = changeCount.get();
// This protects the segmentInfos we are now going
// to commit. This is important in case, eg, while
@@ -2775,7 +2775,7 @@ public class IndexWriter implements Clos
*/
public final synchronized void setCommitData(Map<String,String> commitUserData) {
segmentInfos.setUserData(new HashMap<>(commitUserData));
- ++changeCount;
+ changeCount.incrementAndGet();
}
/**
@@ -2831,7 +2831,7 @@ public class IndexWriter implements Clos
* merged finished, this method may return true right
* after you had just called {@link #commit}. */
public final boolean hasUncommittedChanges() {
- return changeCount != lastCommitChangeCount || docWriter.anyChanges() || bufferedUpdatesStream.any();
+ return changeCount.get() != lastCommitChangeCount || docWriter.anyChanges() || bufferedUpdatesStream.any();
}
private final void commitInternal(MergePolicy mergePolicy) throws IOException {
@@ -4258,7 +4258,7 @@ public class IndexWriter implements Clos
synchronized(this) {
- if (lastCommitChangeCount > changeCount) {
+ if (lastCommitChangeCount > changeCount.get()) {
throw new IllegalStateException("lastCommitChangeCount=" + lastCommitChangeCount + ",changeCount=" + changeCount);
}
Modified: lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestSnapshotDeletionPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestSnapshotDeletionPolicy.java?rev=1671253&r1=1671252&r2=1671253&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestSnapshotDeletionPolicy.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestSnapshotDeletionPolicy.java Sat Apr 4 08:36:52 2015
@@ -22,6 +22,7 @@ import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.Random;
+import java.util.concurrent.CountDownLatch;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.Document;
@@ -293,12 +294,14 @@ public class TestSnapshotDeletionPolicy
Thread[] threads = new Thread[10];
final IndexCommit[] snapshots = new IndexCommit[threads.length];
+ final CountDownLatch startingGun = new CountDownLatch(1);
for (int i = 0; i < threads.length; i++) {
final int finalI = i;
threads[i] = new Thread() {
@Override
public void run() {
try {
+ startingGun.await();
writer.addDocument(new Document());
writer.commit();
snapshots[finalI] = sdp.snapshot();
@@ -309,11 +312,13 @@ public class TestSnapshotDeletionPolicy
};
threads[i].setName("t" + i);
}
-
+
for (Thread t : threads) {
t.start();
}
+ startingGun.countDown();
+
for (Thread t : threads) {
t.join();
}