From 4ee0b4942990d4491a19e05d0ae35db26254a6db Mon Sep 17 00:00:00 2001
From: Mike McCandless
Date: Wed, 25 May 2016 18:41:21 -0400
Subject: [PATCH] sequence numbers: resolve last nocommits
---
.../index/DocumentsWriterDeleteQueue.java | 37 ++-----------------
.../index/DocumentsWriterFlushControl.java | 4 +-
.../org/apache/lucene/index/IndexWriter.java | 20 ++++++++--
.../ControlledRealTimeReopenThread.java | 3 --
.../index/TestIndexingSequenceNumbers.java | 25 +++++--------
5 files changed, 31 insertions(+), 58 deletions(-)
diff --git a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java
index 5d0e83db3cd..abb735d7456 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java
@@ -150,41 +150,10 @@ final class DocumentsWriterDeleteQueue implements Accountable {
return seqNo;
}
- // nocommit can we remove the sync'd
synchronized long add(Node> newNode) {
- /*
- * this non-blocking / 'wait-free' linked list add was inspired by Apache
- * Harmony's ConcurrentLinkedQueue Implementation.
- */
- while (true) {
- final Node> currentTail = this.tail;
- final Node> tailNext = currentTail.next;
- if (tail == currentTail) {
- if (tailNext != null) {
- /*
- * we are in intermediate state here. the tails next pointer has been
- * advanced but the tail itself might not be updated yet. help to
- * advance the tail and try again updating it.
- */
- tailUpdater.compareAndSet(this, currentTail, tailNext); // can fail
- } else {
- /*
- * we are in quiescent state and can try to insert the new node to the
- * current tail if we fail to insert we just retry the operation since
- * somebody else has already added its item
- */
- if (currentTail.casNext(null, newNode)) {
- /*
- * now that we are done we need to advance the tail while another
- * thread could have advanced it already so we can ignore the return
- * type of this CAS call
- */
- tailUpdater.compareAndSet(this, currentTail, newNode);
- return seqNo.getAndIncrement();
- }
- }
- }
- }
+ tail.next = newNode;
+ tail = newNode;
+ return seqNo.getAndIncrement();
}
boolean anyChanges() {
diff --git a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java
index f388f46853a..bd8015dd5c7 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java
@@ -484,8 +484,8 @@ final class DocumentsWriterFlushControl implements Accountable {
// jump over any possible in flight ops:
seqNo = documentsWriter.deleteQueue.seqNo.get() + perThreadPool.getActiveThreadStateCount();
- // nocommit is this (active thread state count) always enough of a gap? what if new indexing thread sneaks in just now? it would
- // have to get this next delete queue?
+ // Insert a gap in seqNo of current active thread count, in the worst case those threads now have one operation in flight. It's fine
+ // if we have some sequence numbers that were never assigned:
DocumentsWriterDeleteQueue newQueue = new DocumentsWriterDeleteQueue(flushingQueue.generation+1, seqNo+1);
documentsWriter.deleteQueue = newQueue;
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 9a520b1dc03..945399c1371 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
@@ -426,7 +426,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
boolean success = false;
synchronized (fullFlushLock) {
try {
- // nocommit should we make this available in the returned NRT reader?
+ // TODO: should we somehow make this available in the returned NRT reader?
long seqNo = docWriter.flushAllThreads();
if (seqNo < 0) {
anyChanges = true;
@@ -2984,7 +2984,11 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
}
startCommit(toCommit);
success = true;
- return seqNo;
+ if (pendingCommit == null) {
+ return -1;
+ } else {
+ return seqNo;
+ }
} finally {
if (!success) {
synchronized (this) {
@@ -3058,6 +3062,12 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
* loss it may still lose data. Lucene cannot guarantee
* consistency on such devices.
*
+ * If nothing was committed, because there were no
+ * pending changes, this returns -1. Otherwise, it returns
+ * the sequence number such that all indexing operations
+ * prior to this sequence will be included in the commit
+ * point, and all other operations will not.
+ *
* @see #prepareCommit
*/
@Override
@@ -4978,9 +4988,11 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
};
}
- // nocommit javadocs
+ /** Returns the last sequence number.
+ *
+ * @lucene.experimental */
public long getLastSequenceNumber() {
ensureOpen();
- return docWriter.deleteQueue.seqNo.get();
+ return docWriter.deleteQueue.seqNo.get()-1;
}
}
diff --git a/lucene/core/src/java/org/apache/lucene/search/ControlledRealTimeReopenThread.java b/lucene/core/src/java/org/apache/lucene/search/ControlledRealTimeReopenThread.java
index d015ae963c1..466d793cc1e 100644
--- a/lucene/core/src/java/org/apache/lucene/search/ControlledRealTimeReopenThread.java
+++ b/lucene/core/src/java/org/apache/lucene/search/ControlledRealTimeReopenThread.java
@@ -151,9 +151,6 @@ public class ControlledRealTimeReopenThread extends Thread implements Closeab
*/
public synchronized boolean waitForGeneration(long targetGen, int maxMS) throws InterruptedException {
final long curGen = writer.getLastSequenceNumber();
- if (targetGen > curGen) {
- throw new IllegalArgumentException("targetGen=" + targetGen + " was never returned by the ReferenceManager instance (current gen=" + curGen + ")");
- }
if (targetGen > searchingGen) {
// Notify the reopen thread that the waitingGen has
// changed, so it may wake up and realize it should
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexingSequenceNumbers.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexingSequenceNumbers.java
index f2e96362675..fb9b9abb7a0 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexingSequenceNumbers.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexingSequenceNumbers.java
@@ -74,8 +74,7 @@ public class TestIndexingSequenceNumbers extends LuceneTestCase {
int iters = atLeast(100);
for(int iter=0;iter