diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java index 47267682017..3889a5d9d23 100644 --- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java +++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java @@ -255,7 +255,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { 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 rollbackSegments; // list of segmentInfo we will fallback to if the commit fails @@ -1526,7 +1526,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { // 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); } @@ -1974,7 +1974,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { deleter.checkpoint(segmentInfos, false); deleter.refresh(); - lastCommitChangeCount = changeCount; + lastCommitChangeCount = changeCount.get(); deleter.close(); @@ -2089,7 +2089,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { // 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(); @@ -2202,13 +2202,13 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { * 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(); } @@ -2712,7 +2712,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { // 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 @@ -2770,7 +2770,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { */ public final synchronized void setCommitData(Map commitUserData) { segmentInfos.setUserData(new HashMap<>(commitUserData)); - ++changeCount; + changeCount.incrementAndGet(); } /** @@ -2826,7 +2826,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { * 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 { @@ -4253,7 +4253,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { synchronized(this) { - if (lastCommitChangeCount > changeCount) { + if (lastCommitChangeCount > changeCount.get()) { throw new IllegalStateException("lastCommitChangeCount=" + lastCommitChangeCount + ",changeCount=" + changeCount); } diff --git a/lucene/core/src/test/org/apache/lucene/index/TestSnapshotDeletionPolicy.java b/lucene/core/src/test/org/apache/lucene/index/TestSnapshotDeletionPolicy.java index dbfd7d95178..1780494b4be 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestSnapshotDeletionPolicy.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestSnapshotDeletionPolicy.java @@ -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 extends LuceneTestCase { 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 extends LuceneTestCase { }; threads[i].setName("t" + i); } - + for (Thread t : threads) { t.start(); } + startingGun.countDown(); + for (Thread t : threads) { t.join(); }