LUCENE-3546: IW#nrtIsCurrent retruns true if changes are in del queue but not in bufferedDeleteStream yet

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1196211 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Simon Willnauer 2011-11-01 19:03:04 +00:00
parent c28be02238
commit 1ce1b914d3
3 changed files with 36 additions and 1 deletions

View File

@ -578,6 +578,7 @@ final class DocumentsWriter {
}
applyFlushTickets();
}
assert !flushingDeleteQueue.anyChanges();
} finally {
assert flushingDeleteQueue == currentFullFlushDelQueue;
}

View File

@ -169,7 +169,13 @@ final class DocumentsWriterDeleteQueue {
boolean anyChanges() {
globalBufferLock.lock();
try {
return !globalSlice.isEmpty() || globalBufferedDeletes.any();
/*
* check if all items in the global slice were applied
* and if the global slice is up-to-date
* and if globalBufferedDeletes has changes
*/
return globalBufferedDeletes.any() || !globalSlice.isEmpty() || globalSlice.sliceTail != tail
|| tail.next != null;
} finally {
globalBufferLock.unlock();
}

View File

@ -16,10 +16,12 @@ package org.apache.lucene.index;
* License for the specific language governing permissions and limitations under
* the License.
*/
import java.lang.reflect.Field;
import java.util.HashSet;
import java.util.Set;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.locks.ReentrantLock;
import org.apache.lucene.index.DocumentsWriterDeleteQueue.DeleteSlice;
import org.apache.lucene.search.TermQuery;
@ -142,6 +144,32 @@ public class TestDocumentsWriterDeleteQueue extends LuceneTestCase {
}
}
}
public void testPartiallyAppliedGlobalSlice() throws SecurityException,
NoSuchFieldException, IllegalArgumentException, IllegalAccessException,
InterruptedException {
final DocumentsWriterDeleteQueue queue = new DocumentsWriterDeleteQueue();
Field field = DocumentsWriterDeleteQueue.class
.getDeclaredField("globalBufferLock");
field.setAccessible(true);
ReentrantLock lock = (ReentrantLock) field.get(queue);
lock.lock();
Thread t = new Thread() {
public void run() {
queue.addDelete(new Term("foo", "bar"));
}
};
t.start();
t.join();
lock.unlock();
assertTrue("changes in del queue but not in slice yet", queue.anyChanges());
queue.tryApplyGlobalSlice();
assertTrue("changes in global buffer", queue.anyChanges());
FrozenBufferedDeletes freezeGlobalBuffer = queue.freezeGlobalBuffer(null);
assertTrue(freezeGlobalBuffer.any());
assertEquals(1, freezeGlobalBuffer.termCount);
assertFalse("all changes applied", queue.anyChanges());
}
public void testStressDeleteQueue() throws InterruptedException {
DocumentsWriterDeleteQueue queue = new DocumentsWriterDeleteQueue();