This commit is contained in:
Karl Wright 2016-06-07 07:11:01 -04:00
commit d74db527bf
48 changed files with 1944 additions and 631 deletions

View File

@ -30,6 +30,10 @@ New Features
applicable and supported when copying files from another FSDirectory in
Directory#copyFrom. (Simon Willnauer)
* LUCENE-7302: IndexWriter methods that change the index now return a
long "sequence number" indicating the effective equivalent
single-threaded execution order (Mike McCandless)
API Changes
* LUCENE-7184: Refactor LatLonPoint encoding methods to new GeoEncodingUtils
@ -144,6 +148,11 @@ Bug Fixes
* LUCENE-7312: Fix geo3d's x/y/z double to int encoding to ensure it always
rounds down (Karl Wright, Mike McCandless)
* LUCENE-7132: BooleanQuery sometimes assigned too-low scores in cases
where ranges of documents had only a single clause matching while
other ranges had more than one clause matching (Ahmet Arslan,
hossman, Mike McCandless)
Documentation
* LUCENE-7223: Improve XXXPoint javadocs to make it clear that you

View File

@ -34,7 +34,7 @@ import org.apache.lucene.util.RamUsageEstimator;
* single segment. This is used to hold buffered pending
* deletes and updates against the to-be-flushed segment. Once the
* deletes and updates are pushed (on flush in DocumentsWriter), they
* are converted to a FrozenDeletes instance. */
* are converted to a FrozenBufferedUpdates instance. */
// NOTE: instances of this class are accessed either via a private
// instance on DocumentWriterPerThread, or via sync'd code by
@ -158,9 +158,12 @@ class BufferedUpdates {
private final static boolean VERBOSE_DELETES = false;
long gen;
final String segmentName;
public BufferedUpdates() {
public BufferedUpdates(String segmentName) {
this.bytesUsed = new AtomicLong();
this.segmentName = segmentName;
}
@Override

View File

@ -136,30 +136,40 @@ final class DocumentsWriter implements Closeable, Accountable {
flushControl = new DocumentsWriterFlushControl(this, config, writer.bufferedUpdatesStream);
}
synchronized boolean deleteQueries(final Query... queries) throws IOException {
synchronized long deleteQueries(final Query... queries) throws IOException {
// TODO why is this synchronized?
final DocumentsWriterDeleteQueue deleteQueue = this.deleteQueue;
deleteQueue.addDelete(queries);
long seqNo = deleteQueue.addDelete(queries);
flushControl.doOnDelete();
return applyAllDeletes(deleteQueue);
if (applyAllDeletes(deleteQueue)) {
seqNo = -seqNo;
}
return seqNo;
}
// TODO: we could check w/ FreqProxTermsWriter: if the
// term doesn't exist, don't bother buffering into the
// per-DWPT map (but still must go into the global map)
synchronized boolean deleteTerms(final Term... terms) throws IOException {
synchronized long deleteTerms(final Term... terms) throws IOException {
// TODO why is this synchronized?
final DocumentsWriterDeleteQueue deleteQueue = this.deleteQueue;
deleteQueue.addDelete(terms);
long seqNo = deleteQueue.addDelete(terms);
flushControl.doOnDelete();
return applyAllDeletes( deleteQueue);
if (applyAllDeletes(deleteQueue)) {
seqNo = -seqNo;
}
return seqNo;
}
synchronized boolean updateDocValues(DocValuesUpdate... updates) throws IOException {
synchronized long updateDocValues(DocValuesUpdate... updates) throws IOException {
final DocumentsWriterDeleteQueue deleteQueue = this.deleteQueue;
deleteQueue.addDocValuesUpdates(updates);
long seqNo = deleteQueue.addDocValuesUpdates(updates);
flushControl.doOnDelete();
return applyAllDeletes(deleteQueue);
if (applyAllDeletes(deleteQueue)) {
seqNo = -seqNo;
}
return seqNo;
}
DocumentsWriterDeleteQueue currentDeleteSession() {
@ -247,6 +257,10 @@ final class DocumentsWriter implements Closeable, Accountable {
abortedDocCount += abortThreadState(perThread);
}
deleteQueue.clear();
// jump over any possible in flight ops:
deleteQueue.skipSequenceNumbers(perThreadPool.getActiveThreadStateCount()+1);
flushControl.abortPendingFlushes();
flushControl.waitForFlush();
success = true;
@ -393,13 +407,14 @@ final class DocumentsWriter implements Closeable, Accountable {
}
}
boolean updateDocuments(final Iterable<? extends Iterable<? extends IndexableField>> docs, final Analyzer analyzer,
final Term delTerm) throws IOException, AbortingException {
long updateDocuments(final Iterable<? extends Iterable<? extends IndexableField>> docs, final Analyzer analyzer,
final Term delTerm) throws IOException, AbortingException {
boolean hasEvents = preUpdate();
final ThreadState perThread = flushControl.obtainAndLock();
final DocumentsWriterPerThread flushingDWPT;
final long seqNo;
try {
// This must happen after we've pulled the ThreadState because IW.close
// waits for all ThreadStates to be released:
@ -409,7 +424,7 @@ final class DocumentsWriter implements Closeable, Accountable {
final DocumentsWriterPerThread dwpt = perThread.dwpt;
final int dwptNumDocs = dwpt.getNumDocsInRAM();
try {
dwpt.updateDocuments(docs, analyzer, delTerm);
seqNo = dwpt.updateDocuments(docs, analyzer, delTerm);
} catch (AbortingException ae) {
flushControl.doOnAbort(perThread);
dwpt.abort();
@ -426,10 +441,14 @@ final class DocumentsWriter implements Closeable, Accountable {
perThreadPool.release(perThread);
}
return postUpdate(flushingDWPT, hasEvents);
if (postUpdate(flushingDWPT, hasEvents)) {
return -seqNo;
} else {
return seqNo;
}
}
boolean updateDocument(final Iterable<? extends IndexableField> doc, final Analyzer analyzer,
long updateDocument(final Iterable<? extends IndexableField> doc, final Analyzer analyzer,
final Term delTerm) throws IOException, AbortingException {
boolean hasEvents = preUpdate();
@ -437,6 +456,7 @@ final class DocumentsWriter implements Closeable, Accountable {
final ThreadState perThread = flushControl.obtainAndLock();
final DocumentsWriterPerThread flushingDWPT;
final long seqNo;
try {
// This must happen after we've pulled the ThreadState because IW.close
// waits for all ThreadStates to be released:
@ -446,7 +466,7 @@ final class DocumentsWriter implements Closeable, Accountable {
final DocumentsWriterPerThread dwpt = perThread.dwpt;
final int dwptNumDocs = dwpt.getNumDocsInRAM();
try {
dwpt.updateDocument(doc, analyzer, delTerm);
seqNo = dwpt.updateDocument(doc, analyzer, delTerm);
} catch (AbortingException ae) {
flushControl.doOnAbort(perThread);
dwpt.abort();
@ -463,7 +483,11 @@ final class DocumentsWriter implements Closeable, Accountable {
perThreadPool.release(perThread);
}
return postUpdate(flushingDWPT, hasEvents);
if (postUpdate(flushingDWPT, hasEvents)) {
return -seqNo;
} else {
return seqNo;
}
}
private boolean doFlush(DocumentsWriterPerThread flushingDWPT) throws IOException, AbortingException {
@ -587,20 +611,22 @@ final class DocumentsWriter implements Closeable, Accountable {
* two stage operation; the caller must ensure (in try/finally) that finishFlush
* is called after this method, to release the flush lock in DWFlushControl
*/
boolean flushAllThreads()
long flushAllThreads()
throws IOException, AbortingException {
final DocumentsWriterDeleteQueue flushingDeleteQueue;
if (infoStream.isEnabled("DW")) {
infoStream.message("DW", "startFullFlush");
}
long seqNo;
synchronized (this) {
pendingChangesInCurrentFullFlush = anyChanges();
flushingDeleteQueue = deleteQueue;
/* Cutover to a new delete queue. This must be synced on the flush control
* otherwise a new DWPT could sneak into the loop with an already flushing
* delete queue */
flushControl.markForFullFlush(); // swaps the delQueue synced on FlushControl
seqNo = flushControl.markForFullFlush(); // swaps this.deleteQueue synced on FlushControl
assert setFlushingDeleteQueue(flushingDeleteQueue);
}
assert currentFullFlushDelQueue != null;
@ -620,13 +646,17 @@ final class DocumentsWriter implements Closeable, Accountable {
infoStream.message("DW", Thread.currentThread().getName() + ": flush naked frozen global deletes");
}
ticketQueue.addDeletes(flushingDeleteQueue);
}
}
ticketQueue.forcePurge(writer);
assert !flushingDeleteQueue.anyChanges() && !ticketQueue.hasTickets();
} finally {
assert flushingDeleteQueue == currentFullFlushDelQueue;
}
return anythingFlushed;
if (anythingFlushed) {
return -seqNo;
} else {
return seqNo;
}
}
void finishFullFlush(IndexWriter indexWriter, boolean success) {
@ -646,7 +676,6 @@ final class DocumentsWriter implements Closeable, Accountable {
} finally {
pendingChangesInCurrentFullFlush = false;
}
}
public LiveIndexWriterConfig getIndexWriterConfig() {

View File

@ -17,6 +17,7 @@
package org.apache.lucene.index;
import java.util.Arrays;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
import java.util.concurrent.locks.ReentrantLock;
@ -68,12 +69,15 @@ import org.apache.lucene.util.BytesRef;
*/
final class DocumentsWriterDeleteQueue implements Accountable {
// the current end (latest delete operation) in the delete queue:
private volatile Node<?> tail;
@SuppressWarnings("rawtypes")
private static final AtomicReferenceFieldUpdater<DocumentsWriterDeleteQueue,Node> tailUpdater = AtomicReferenceFieldUpdater
.newUpdater(DocumentsWriterDeleteQueue.class, Node.class, "tail");
/** Used to record deletes against all prior (already written to disk) segments. Whenever any segment flushes, we bundle up this set of
* deletes and insert into the buffered updates stream before the newly flushed segment(s). */
private final DeleteSlice globalSlice;
private final BufferedUpdates globalBufferedUpdates;
@ -81,18 +85,26 @@ final class DocumentsWriterDeleteQueue implements Accountable {
final ReentrantLock globalBufferLock = new ReentrantLock();
final long generation;
/** Generates the sequence number that IW returns to callers changing the index, showing the effective serialization of all operations. */
private final AtomicLong nextSeqNo;
// for asserts
long maxSeqNo = Long.MAX_VALUE;
DocumentsWriterDeleteQueue() {
this(0);
// seqNo must start at 1 because some APIs negate this to also return a boolean
this(0, 1);
}
DocumentsWriterDeleteQueue(long generation) {
this(new BufferedUpdates(), generation);
DocumentsWriterDeleteQueue(long generation, long startSeqNo) {
this(new BufferedUpdates("global"), generation, startSeqNo);
}
DocumentsWriterDeleteQueue(BufferedUpdates globalBufferedUpdates, long generation) {
DocumentsWriterDeleteQueue(BufferedUpdates globalBufferedUpdates, long generation, long startSeqNo) {
this.globalBufferedUpdates = globalBufferedUpdates;
this.generation = generation;
this.nextSeqNo = new AtomicLong(startSeqNo);
/*
* we use a sentinel instance as our initial tail. No slice will ever try to
* apply this tail since the head is always omitted.
@ -101,28 +113,30 @@ final class DocumentsWriterDeleteQueue implements Accountable {
globalSlice = new DeleteSlice(tail);
}
void addDelete(Query... queries) {
add(new QueryArrayNode(queries));
long addDelete(Query... queries) {
long seqNo = add(new QueryArrayNode(queries));
tryApplyGlobalSlice();
return seqNo;
}
void addDelete(Term... terms) {
add(new TermArrayNode(terms));
long addDelete(Term... terms) {
long seqNo = add(new TermArrayNode(terms));
tryApplyGlobalSlice();
return seqNo;
}
void addDocValuesUpdates(DocValuesUpdate... updates) {
add(new DocValuesUpdatesNode(updates));
long addDocValuesUpdates(DocValuesUpdate... updates) {
long seqNo = add(new DocValuesUpdatesNode(updates));
tryApplyGlobalSlice();
return seqNo;
}
/**
* invariant for document update
*/
void add(Term term, DeleteSlice slice) {
long add(Term term, DeleteSlice slice) {
final TermNode termNode = new TermNode(term);
// System.out.println(Thread.currentThread().getName() + ": push " + termNode + " this=" + this);
add(termNode);
long seqNo = add(termNode);
/*
* this is an update request where the term is the updated documents
* delTerm. in that case we need to guarantee that this insert is atomic
@ -137,42 +151,14 @@ final class DocumentsWriterDeleteQueue implements Accountable {
assert slice.sliceHead != slice.sliceTail : "slice head and tail must differ after add";
tryApplyGlobalSlice(); // TODO doing this each time is not necessary maybe
// we can do it just every n times or so?
return seqNo;
}
void add(Node<?> item) {
/*
* 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 item 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, item)) {
/*
* 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, item);
return;
}
}
}
}
synchronized long add(Node<?> newNode) {
tail.next = newNode;
this.tail = newNode;
return getNextSequenceNumber();
}
boolean anyChanges() {
@ -183,8 +169,7 @@ final class DocumentsWriterDeleteQueue implements Accountable {
* and if the global slice is up-to-date
* and if globalBufferedUpdates has changes
*/
return globalBufferedUpdates.any() || !globalSlice.isEmpty() || globalSlice.sliceTail != tail
|| tail.next != null;
return globalBufferedUpdates.any() || !globalSlice.isEmpty() || globalSlice.sliceTail != tail || tail.next != null;
} finally {
globalBufferLock.unlock();
}
@ -199,8 +184,7 @@ final class DocumentsWriterDeleteQueue implements Accountable {
* tail the next time we can get the lock!
*/
try {
if (updateSlice(globalSlice)) {
// System.out.println(Thread.currentThread() + ": apply globalSlice");
if (updateSliceNoSeqNo(globalSlice)) {
globalSlice.apply(globalBufferedUpdates, BufferedUpdates.MAX_INT);
}
} finally {
@ -229,9 +213,7 @@ final class DocumentsWriterDeleteQueue implements Accountable {
globalSlice.apply(globalBufferedUpdates, BufferedUpdates.MAX_INT);
}
// System.out.println(Thread.currentThread().getName() + ": now freeze global buffer " + globalBufferedDeletes);
final FrozenBufferedUpdates packet = new FrozenBufferedUpdates(
globalBufferedUpdates, false);
final FrozenBufferedUpdates packet = new FrozenBufferedUpdates(globalBufferedUpdates, false);
globalBufferedUpdates.clear();
return packet;
} finally {
@ -243,8 +225,21 @@ final class DocumentsWriterDeleteQueue implements Accountable {
return new DeleteSlice(tail);
}
boolean updateSlice(DeleteSlice slice) {
if (slice.sliceTail != tail) { // If we are the same just
/** Negative result means there were new deletes since we last applied */
synchronized long updateSlice(DeleteSlice slice) {
long seqNo = getNextSequenceNumber();
if (slice.sliceTail != tail) {
// new deletes arrived since we last checked
slice.sliceTail = tail;
seqNo = -seqNo;
}
return seqNo;
}
/** Just like updateSlice, but does not assign a sequence number */
boolean updateSliceNoSeqNo(DeleteSlice slice) {
if (slice.sliceTail != tail) {
// new deletes arrived since we last checked
slice.sliceTail = tail;
return true;
}
@ -282,7 +277,6 @@ final class DocumentsWriterDeleteQueue implements Accountable {
current = current.next;
assert current != null : "slice property violated between the head on the tail must not be a null node";
current.apply(del, docIDUpto);
// System.out.println(Thread.currentThread().getName() + ": pull " + current + " docIDUpto=" + docIDUpto);
} while (current != sliceTail);
reset();
}
@ -459,6 +453,20 @@ final class DocumentsWriterDeleteQueue implements Accountable {
public String toString() {
return "DWDQ: [ generation: " + generation + " ]";
}
public long getNextSequenceNumber() {
long seqNo = nextSeqNo.getAndIncrement();
assert seqNo < maxSeqNo: "seqNo=" + seqNo + " vs maxSeqNo=" + maxSeqNo;
return seqNo;
}
public long getLastSequenceNumber() {
return nextSeqNo.get()-1;
}
/** Inserts a gap in the sequence numbers. This is used by IW during flush or commit to ensure any in-flight threads get sequence numbers
* inside the gap */
public void skipSequenceNumbers(long jump) {
nextSeqNo.addAndGet(jump);
}
}

View File

@ -141,8 +141,7 @@ final class DocumentsWriterFlushControl implements Accountable {
}
private void commitPerThreadBytes(ThreadState perThread) {
final long delta = perThread.dwpt.bytesUsed()
- perThread.bytesUsed;
final long delta = perThread.dwpt.bytesUsed() - perThread.bytesUsed;
perThread.bytesUsed += delta;
/*
* We need to differentiate here if we are pending since setFlushPending
@ -167,8 +166,7 @@ final class DocumentsWriterFlushControl implements Accountable {
return true;
}
synchronized DocumentsWriterPerThread doAfterDocument(ThreadState perThread,
boolean isUpdate) {
synchronized DocumentsWriterPerThread doAfterDocument(ThreadState perThread, boolean isUpdate) {
try {
commitPerThreadBytes(perThread);
if (!perThread.flushPending) {
@ -192,7 +190,7 @@ final class DocumentsWriterFlushControl implements Accountable {
flushingDWPT = null;
}
} else {
flushingDWPT = tryCheckoutForFlush(perThread);
flushingDWPT = tryCheckoutForFlush(perThread);
}
return flushingDWPT;
} finally {
@ -454,8 +452,7 @@ final class DocumentsWriterFlushControl implements Accountable {
.currentThread(), documentsWriter);
boolean success = false;
try {
if (perThread.isInitialized()
&& perThread.dwpt.deleteQueue != documentsWriter.deleteQueue) {
if (perThread.isInitialized() && perThread.dwpt.deleteQueue != documentsWriter.deleteQueue) {
// There is a flush-all in process and this DWPT is
// now stale -- enroll it for flush and try for
// another DWPT:
@ -471,8 +468,9 @@ final class DocumentsWriterFlushControl implements Accountable {
}
}
void markForFullFlush() {
long markForFullFlush() {
final DocumentsWriterDeleteQueue flushingQueue;
long seqNo;
synchronized (this) {
assert !fullFlush : "called DWFC#markForFullFlush() while full flush is still running";
assert fullFlushBuffer.isEmpty() : "full flush buffer should be empty: "+ fullFlushBuffer;
@ -480,7 +478,14 @@ final class DocumentsWriterFlushControl implements Accountable {
flushingQueue = documentsWriter.deleteQueue;
// Set a new delete queue - all subsequent DWPT will use this queue until
// we do another full flush
DocumentsWriterDeleteQueue newQueue = new DocumentsWriterDeleteQueue(flushingQueue.generation+1);
// Insert a gap in seqNo of current active thread count, in the worst case each of those threads now have one operation in flight. It's fine
// if we have some sequence numbers that were never assigned:
seqNo = documentsWriter.deleteQueue.getLastSequenceNumber() + perThreadPool.getActiveThreadStateCount() + 2;
flushingQueue.maxSeqNo = seqNo+1;
DocumentsWriterDeleteQueue newQueue = new DocumentsWriterDeleteQueue(flushingQueue.generation+1, seqNo+1);
documentsWriter.deleteQueue = newQueue;
}
final int limit = perThreadPool.getActiveThreadStateCount();
@ -520,6 +525,7 @@ final class DocumentsWriterFlushControl implements Accountable {
updateStallState();
}
assert assertActiveDeleteQueue(documentsWriter.deleteQueue);
return seqNo;
}
private boolean assertActiveDeleteQueue(DocumentsWriterDeleteQueue queue) {

View File

@ -171,11 +171,10 @@ class DocumentsWriterPerThread {
this.pendingNumDocs = pendingNumDocs;
bytesUsed = Counter.newCounter();
byteBlockAllocator = new DirectTrackingAllocator(bytesUsed);
pendingUpdates = new BufferedUpdates();
pendingUpdates = new BufferedUpdates(segmentName);
intBlockAllocator = new IntBlockAllocator(bytesUsed);
this.deleteQueue = deleteQueue;
assert numDocsInRAM == 0 : "num docs " + numDocsInRAM;
pendingUpdates.clear();
deleteSlice = deleteQueue.newSlice();
segmentInfo = new SegmentInfo(directoryOrig, Version.LATEST, segmentName, -1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null);
@ -210,7 +209,7 @@ class DocumentsWriterPerThread {
}
}
public void updateDocument(Iterable<? extends IndexableField> doc, Analyzer analyzer, Term delTerm) throws IOException, AbortingException {
public long updateDocument(Iterable<? extends IndexableField> doc, Analyzer analyzer, Term delTerm) throws IOException, AbortingException {
testPoint("DocumentsWriterPerThread addDocument start");
assert deleteQueue != null;
reserveOneDoc();
@ -241,10 +240,11 @@ class DocumentsWriterPerThread {
numDocsInRAM++;
}
}
finishDocument(delTerm);
return finishDocument(delTerm);
}
public int updateDocuments(Iterable<? extends Iterable<? extends IndexableField>> docs, Analyzer analyzer, Term delTerm) throws IOException, AbortingException {
public long updateDocuments(Iterable<? extends Iterable<? extends IndexableField>> docs, Analyzer analyzer, Term delTerm) throws IOException, AbortingException {
testPoint("DocumentsWriterPerThread addDocuments start");
assert deleteQueue != null;
docState.analyzer = analyzer;
@ -278,19 +278,32 @@ class DocumentsWriterPerThread {
numDocsInRAM++;
}
}
finishDocument(null);
numDocsInRAM++;
}
allDocsIndexed = true;
// Apply delTerm only after all indexing has
// succeeded, but apply it only to docs prior to when
// this batch started:
long seqNo;
if (delTerm != null) {
deleteQueue.add(delTerm, deleteSlice);
seqNo = deleteQueue.add(delTerm, deleteSlice);
assert deleteSlice.isTailItem(delTerm) : "expected the delete term as the tail item";
deleteSlice.apply(pendingUpdates, numDocsInRAM-docCount);
return seqNo;
} else {
seqNo = deleteQueue.updateSlice(deleteSlice);
if (seqNo < 0) {
seqNo = -seqNo;
deleteSlice.apply(pendingUpdates, numDocsInRAM-docCount);
} else {
deleteSlice.reset();
}
}
return seqNo;
} finally {
if (!allDocsIndexed && !aborted) {
// the iterator threw an exception that is not aborting
@ -304,11 +317,9 @@ class DocumentsWriterPerThread {
}
docState.clear();
}
return docCount;
}
private void finishDocument(Term delTerm) {
private long finishDocument(Term delTerm) {
/*
* here we actually finish the document in two steps 1. push the delete into
* the queue and update our slice. 2. increment the DWPT private document
@ -318,11 +329,18 @@ class DocumentsWriterPerThread {
* since we updated the slice the last time.
*/
boolean applySlice = numDocsInRAM != 0;
long seqNo;
if (delTerm != null) {
deleteQueue.add(delTerm, deleteSlice);
seqNo = deleteQueue.add(delTerm, deleteSlice);
assert deleteSlice.isTailItem(delTerm) : "expected the delete term as the tail item";
} else {
applySlice &= deleteQueue.updateSlice(deleteSlice);
seqNo = deleteQueue.updateSlice(deleteSlice);
if (seqNo < 0) {
seqNo = -seqNo;
} else {
applySlice = false;
}
}
if (applySlice) {
@ -331,6 +349,8 @@ class DocumentsWriterPerThread {
deleteSlice.reset();
}
++numDocsInRAM;
return seqNo;
}
// Buffer a specific docID for deletion. Currently only

View File

@ -226,22 +226,8 @@ final class DocumentsWriterPerThreadPool {
return threadStates.get(ord);
}
// TODO: merge this with getActiveThreadStateCount: they are the same!
synchronized int getMaxThreadStates() {
return threadStates.size();
}
/**
* Returns the ThreadState with the minimum estimated number of threads
* waiting to acquire its lock or <code>null</code> if no {@link ThreadState}
* is yet visible to the calling thread.
*/
ThreadState minContendedThreadState() {
ThreadState minThreadState = null;
for (ThreadState state : threadStates) {
if (minThreadState == null || state.getQueueLength() < minThreadState.getQueueLength()) {
minThreadState = state;
}
}
return minThreadState;
}
}

View File

@ -95,6 +95,14 @@ import org.apache.lucene.util.Version;
and then adds the entire document). When finished adding, deleting
and updating documents, {@link #close() close} should be called.</p>
<a name="sequence_numbers"></a>
<p>Each method that changes the index returns a {@code long} sequence number, which
expresses the effective order in which each change was applied.
{@link #commit} also returns a sequence number, describing which
changes are in the commit point and which are not. Sequence numbers
are transient (not saved into the index in any way) and only valid
within a single {@code IndexWriter} instance.</p>
<a name="flush"></a>
<p>These changes are buffered in memory and periodically
flushed to the {@link Directory} (during the above method
@ -266,6 +274,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
private List<SegmentCommitInfo> rollbackSegments; // list of segmentInfo we will fallback to if the commit fails
volatile SegmentInfos pendingCommit; // set when a commit is pending (after prepareCommit() & before commit())
volatile long pendingSeqNo;
volatile long pendingCommitChangeCount;
private Collection<String> filesToCommit;
@ -425,7 +434,14 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
boolean success = false;
synchronized (fullFlushLock) {
try {
anyChanges = docWriter.flushAllThreads();
// TODO: should we somehow make this available in the returned NRT reader?
long seqNo = docWriter.flushAllThreads();
if (seqNo < 0) {
anyChanges = true;
seqNo = -seqNo;
} else {
anyChanges = false;
}
if (!anyChanges) {
// prevent double increment since docWriter#doFlush increments the flushcount
// if we flushed anything.
@ -1280,11 +1296,14 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
* replaced with the Unicode replacement character
* U+FFFD.</p>
*
* @return The <a href="#sequence_number">sequence number</a>
* for this operation
*
* @throws CorruptIndexException if the index is corrupt
* @throws IOException if there is a low-level IO error
*/
public void addDocument(Iterable<? extends IndexableField> doc) throws IOException {
updateDocument(null, doc);
public long addDocument(Iterable<? extends IndexableField> doc) throws IOException {
return updateDocument(null, doc);
}
/**
@ -1319,13 +1338,16 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
* and will likely break them up. Use such tools at your
* own risk!
*
* @return The <a href="#sequence_number">sequence number</a>
* for this operation
*
* @throws CorruptIndexException if the index is corrupt
* @throws IOException if there is a low-level IO error
*
* @lucene.experimental
*/
public void addDocuments(Iterable<? extends Iterable<? extends IndexableField>> docs) throws IOException {
updateDocuments(null, docs);
public long addDocuments(Iterable<? extends Iterable<? extends IndexableField>> docs) throws IOException {
return updateDocuments(null, docs);
}
/**
@ -1336,20 +1358,26 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
*
* See {@link #addDocuments(Iterable)}.
*
* @return The <a href="#sequence_number">sequence number</a>
* for this operation
*
* @throws CorruptIndexException if the index is corrupt
* @throws IOException if there is a low-level IO error
*
* @lucene.experimental
*/
public void updateDocuments(Term delTerm, Iterable<? extends Iterable<? extends IndexableField>> docs) throws IOException {
public long updateDocuments(Term delTerm, Iterable<? extends Iterable<? extends IndexableField>> docs) throws IOException {
ensureOpen();
try {
boolean success = false;
try {
if (docWriter.updateDocuments(docs, analyzer, delTerm)) {
long seqNo = docWriter.updateDocuments(docs, analyzer, delTerm);
if (seqNo < 0) {
seqNo = -seqNo;
processEvents(true, false);
}
success = true;
return seqNo;
} finally {
if (!success) {
if (infoStream.isEnabled("IW")) {
@ -1359,6 +1387,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
}
} catch (AbortingException | VirtualMachineError tragedy) {
tragicEvent(tragedy, "updateDocuments");
// dead code but javac disagrees
return -1;
}
}
@ -1367,15 +1398,15 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
* DirectoryReader#open(IndexWriter)}). If the
* provided reader is an NRT reader obtained from this
* writer, and its segment has not been merged away, then
* the delete succeeds and this method returns true; else, it
* returns false the caller must then separately delete by
* Term or Query.
* the delete succeeds and this method returns a valid (&gt; 0) sequence
* number; else, it returns -1 and the caller must then
* separately delete by Term or Query.
*
* <b>NOTE</b>: this method can only delete documents
* visible to the currently open NRT reader. If you need
* to delete documents indexed after opening the NRT
* reader you must use {@link #deleteDocuments(Term...)}). */
public synchronized boolean tryDeleteDocument(IndexReader readerIn, int docID) throws IOException {
public synchronized long tryDeleteDocument(IndexReader readerIn, int docID) throws IOException {
final LeafReader reader;
if (readerIn instanceof LeafReader) {
@ -1426,7 +1457,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
changed();
}
//System.out.println(" yes " + info.info.name + " " + docID);
return true;
return docWriter.deleteQueue.getNextSequenceNumber();
}
} else {
//System.out.println(" no rld " + info.info.name + " " + docID);
@ -1434,7 +1466,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
} else {
//System.out.println(" no seg " + info.info.name + " " + docID);
}
return false;
return -1;
}
/**
@ -1442,19 +1475,28 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
* terms. All given deletes are applied and flushed atomically
* at the same time.
*
* @return The <a href="#sequence_number">sequence number</a>
* for this operation
*
* @param terms array of terms to identify the documents
* to be deleted
* @throws CorruptIndexException if the index is corrupt
* @throws IOException if there is a low-level IO error
*/
public void deleteDocuments(Term... terms) throws IOException {
public long deleteDocuments(Term... terms) throws IOException {
ensureOpen();
try {
if (docWriter.deleteTerms(terms)) {
long seqNo = docWriter.deleteTerms(terms);
if (seqNo < 0) {
seqNo = -seqNo;
processEvents(true, false);
}
return seqNo;
} catch (VirtualMachineError tragedy) {
tragicEvent(tragedy, "deleteDocuments(Term..)");
// dead code but javac disagrees:
return -1;
}
}
@ -1462,28 +1504,37 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
* Deletes the document(s) matching any of the provided queries.
* All given deletes are applied and flushed atomically at the same time.
*
* @return The <a href="#sequence_number">sequence number</a>
* for this operation
*
* @param queries array of queries to identify the documents
* to be deleted
* @throws CorruptIndexException if the index is corrupt
* @throws IOException if there is a low-level IO error
*/
public void deleteDocuments(Query... queries) throws IOException {
public long deleteDocuments(Query... queries) throws IOException {
ensureOpen();
// LUCENE-6379: Specialize MatchAllDocsQuery
for(Query query : queries) {
if (query.getClass() == MatchAllDocsQuery.class) {
deleteAll();
return;
return deleteAll();
}
}
try {
if (docWriter.deleteQueries(queries)) {
long seqNo = docWriter.deleteQueries(queries);
if (seqNo < 0) {
seqNo = -seqNo;
processEvents(true, false);
}
return seqNo;
} catch (VirtualMachineError tragedy) {
tragicEvent(tragedy, "deleteDocuments(Query..)");
// dead code but javac disagrees:
return -1;
}
}
@ -1494,21 +1545,27 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
* by a reader on the same index (flush may happen only after
* the add).
*
* @return The <a href="#sequence_number">sequence number</a>
* for this operation
*
* @param term the term to identify the document(s) to be
* deleted
* @param doc the document to be added
* @throws CorruptIndexException if the index is corrupt
* @throws IOException if there is a low-level IO error
*/
public void updateDocument(Term term, Iterable<? extends IndexableField> doc) throws IOException {
public long updateDocument(Term term, Iterable<? extends IndexableField> doc) throws IOException {
ensureOpen();
try {
boolean success = false;
try {
if (docWriter.updateDocument(doc, analyzer, term)) {
long seqNo = docWriter.updateDocument(doc, analyzer, term);
if (seqNo < 0) {
seqNo = - seqNo;
processEvents(true, false);
}
success = true;
return seqNo;
} finally {
if (!success) {
if (infoStream.isEnabled("IW")) {
@ -1518,6 +1575,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
}
} catch (AbortingException | VirtualMachineError tragedy) {
tragicEvent(tragedy, "updateDocument");
// dead code but javac disagrees:
return -1;
}
}
@ -1532,22 +1592,32 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
* field name of the {@link NumericDocValues} field
* @param value
* new value for the field
*
* @return The <a href="#sequence_number">sequence number</a>
* for this operation
*
* @throws CorruptIndexException
* if the index is corrupt
* @throws IOException
* if there is a low-level IO error
*/
public void updateNumericDocValue(Term term, String field, long value) throws IOException {
public long updateNumericDocValue(Term term, String field, long value) throws IOException {
ensureOpen();
if (!globalFieldNumberMap.contains(field, DocValuesType.NUMERIC)) {
throw new IllegalArgumentException("can only update existing numeric-docvalues fields!");
}
try {
if (docWriter.updateDocValues(new NumericDocValuesUpdate(term, field, value))) {
long seqNo = docWriter.updateDocValues(new NumericDocValuesUpdate(term, field, value));
if (seqNo < 0) {
seqNo = -seqNo;
processEvents(true, false);
}
return seqNo;
} catch (VirtualMachineError tragedy) {
tragicEvent(tragedy, "updateNumericDocValue");
// dead code but javac disagrees:
return -1;
}
}
@ -1566,12 +1636,16 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
* field name of the {@link BinaryDocValues} field
* @param value
* new value for the field
*
* @return The <a href="#sequence_number">sequence number</a>
* for this operation
*
* @throws CorruptIndexException
* if the index is corrupt
* @throws IOException
* if there is a low-level IO error
*/
public void updateBinaryDocValue(Term term, String field, BytesRef value) throws IOException {
public long updateBinaryDocValue(Term term, String field, BytesRef value) throws IOException {
ensureOpen();
if (value == null) {
throw new IllegalArgumentException("cannot update a field to a null value: " + field);
@ -1580,11 +1654,17 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
throw new IllegalArgumentException("can only update existing binary-docvalues fields!");
}
try {
if (docWriter.updateDocValues(new BinaryDocValuesUpdate(term, field, value))) {
long seqNo = docWriter.updateDocValues(new BinaryDocValuesUpdate(term, field, value));
if (seqNo < 0) {
seqNo = -seqNo;
processEvents(true, false);
}
return seqNo;
} catch (VirtualMachineError tragedy) {
tragicEvent(tragedy, "updateBinaryDocValue");
// dead code but javac disagrees:
return -1;
}
}
@ -1596,12 +1676,16 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
*
* @param updates
* the updates to apply
*
* @return The <a href="#sequence_number">sequence number</a>
* for this operation
*
* @throws CorruptIndexException
* if the index is corrupt
* @throws IOException
* if there is a low-level IO error
*/
public void updateDocValues(Term term, Field... updates) throws IOException {
public long updateDocValues(Term term, Field... updates) throws IOException {
ensureOpen();
DocValuesUpdate[] dvUpdates = new DocValuesUpdate[updates.length];
for (int i = 0; i < updates.length; i++) {
@ -1628,11 +1712,17 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
}
}
try {
if (docWriter.updateDocValues(dvUpdates)) {
long seqNo = docWriter.updateDocValues(dvUpdates);
if (seqNo < 0) {
seqNo = -seqNo;
processEvents(true, false);
}
return seqNo;
} catch (VirtualMachineError tragedy) {
tragicEvent(tragedy, "updateDocValues");
// dead code but javac disagrees:
return -1;
}
}
@ -2204,8 +2294,11 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
* threads are running {@link #forceMerge}, {@link #addIndexes(CodecReader[])}
* or {@link #forceMergeDeletes} methods, they may receive
* {@link MergePolicy.MergeAbortedException}s.
*
* @return The <a href="#sequence_number">sequence number</a>
* for this operation
*/
public void deleteAll() throws IOException {
public long deleteAll() throws IOException {
ensureOpen();
// Remove any buffered docs
boolean success = false;
@ -2252,6 +2345,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
globalFieldNumberMap.clear();
success = true;
return docWriter.deleteQueue.getNextSequenceNumber();
} finally {
docWriter.unlockAllAfterAbortAll(this);
if (!success) {
@ -2264,6 +2359,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
}
} catch (VirtualMachineError tragedy) {
tragicEvent(tragedy, "deleteAll");
// dead code but javac disagrees
return -1;
}
}
@ -2485,13 +2583,16 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
*
* <p>This requires this index not be among those to be added.
*
* @return The <a href="#sequence_number">sequence number</a>
* for this operation
*
* @throws CorruptIndexException if the index is corrupt
* @throws IOException if there is a low-level IO error
* @throws IllegalArgumentException if addIndexes would cause
* the index to exceed {@link #MAX_DOCS}, or if the indoming
* index sort does not match this index's index sort
*/
public void addIndexes(Directory... dirs) throws IOException {
public long addIndexes(Directory... dirs) throws IOException {
ensureOpen();
noDupDirs(dirs);
@ -2502,6 +2603,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
boolean successTop = false;
long seqNo;
try {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "flush at addIndexes(Directory...)");
@ -2573,6 +2676,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
// Now reserve the docs, just before we update SIS:
reserveDocs(totalMaxDoc);
seqNo = docWriter.deleteQueue.getNextSequenceNumber();
success = true;
} finally {
if (!success) {
@ -2590,6 +2695,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
} catch (VirtualMachineError tragedy) {
tragicEvent(tragedy, "addIndexes(Directory...)");
// dead code but javac disagrees:
seqNo = -1;
} finally {
if (successTop) {
IOUtils.close(locks);
@ -2598,6 +2705,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
}
}
maybeMerge();
return seqNo;
}
/**
@ -2622,6 +2731,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
* {@code maxMergeAtOnce} parameter, you should pass that many readers in one
* call.
*
* @return The <a href="#sequence_number">sequence number</a>
* for this operation
*
* @throws CorruptIndexException
* if the index is corrupt
* @throws IOException
@ -2629,7 +2741,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
* @throws IllegalArgumentException
* if addIndexes would cause the index to exceed {@link #MAX_DOCS}
*/
public void addIndexes(CodecReader... readers) throws IOException {
public long addIndexes(CodecReader... readers) throws IOException {
ensureOpen();
// long so we can detect int overflow:
@ -2637,6 +2749,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
Sort indexSort = config.getIndexSort();
long seqNo;
try {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "flush at addIndexes(CodecReader...)");
@ -2671,7 +2785,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
rateLimiters.set(new MergeRateLimiter(null));
if (!merger.shouldMerge()) {
return;
return docWriter.deleteQueue.getNextSequenceNumber();
}
merger.merge(); // merge 'em
@ -2689,7 +2803,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
if (stopMerges) {
// Safe: these files must exist
deleteNewFiles(infoPerCommit.files());
return;
return docWriter.deleteQueue.getNextSequenceNumber();
}
ensureOpen();
useCompoundFile = mergePolicy.useCompoundFile(segmentInfos, infoPerCommit, this);
@ -2724,7 +2839,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
if (stopMerges) {
// Safe: these files must exist
deleteNewFiles(infoPerCommit.files());
return;
return docWriter.deleteQueue.getNextSequenceNumber();
}
ensureOpen();
@ -2732,12 +2848,17 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
reserveDocs(numDocs);
segmentInfos.add(infoPerCommit);
seqNo = docWriter.deleteQueue.getNextSequenceNumber();
checkpoint();
}
} catch (VirtualMachineError tragedy) {
tragicEvent(tragedy, "addIndexes(CodecReader...)");
// dead code but javac disagrees:
seqNo = -1;
}
maybeMerge();
return seqNo;
}
/** Copies the segment files as-is into the IndexWriter's directory. */
@ -2805,14 +2926,19 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
* <p>You can also just call {@link #commit()} directly
* without prepareCommit first in which case that method
* will internally call prepareCommit.
*
* @return The <a href="#sequence_number">sequence number</a>
* of the last operation in the commit. All sequence numbers &lt;= this value
* will be reflected in the commit, and all others will not.
*/
@Override
public final void prepareCommit() throws IOException {
public final long prepareCommit() throws IOException {
ensureOpen();
prepareCommitInternal(config.getMergePolicy());
pendingSeqNo = prepareCommitInternal(config.getMergePolicy());
return pendingSeqNo;
}
private void prepareCommitInternal(MergePolicy mergePolicy) throws IOException {
private long prepareCommitInternal(MergePolicy mergePolicy) throws IOException {
startCommitTime = System.nanoTime();
synchronized(commitLock) {
ensureOpen(false);
@ -2833,6 +2959,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
testPoint("startDoFlush");
SegmentInfos toCommit = null;
boolean anySegmentsFlushed = false;
long seqNo;
// This is copied from doFlush, except it's modified to
// clone & incRef the flushed SegmentInfos inside the
@ -2844,7 +2971,11 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
boolean flushSuccess = false;
boolean success = false;
try {
anySegmentsFlushed = docWriter.flushAllThreads();
seqNo = docWriter.flushAllThreads();
if (seqNo < 0) {
anySegmentsFlushed = true;
seqNo = -seqNo;
}
if (!anySegmentsFlushed) {
// prevent double increment since docWriter#doFlush increments the flushcount
// if we flushed anything.
@ -2898,6 +3029,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
}
} catch (AbortingException | VirtualMachineError tragedy) {
tragicEvent(tragedy, "prepareCommit");
// dead code but javac disagrees:
seqNo = -1;
}
boolean success = false;
@ -2907,6 +3041,11 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
}
startCommit(toCommit);
success = true;
if (pendingCommit == null) {
return -1;
} else {
return seqNo;
}
} finally {
if (!success) {
synchronized (this) {
@ -2980,12 +3119,22 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
* loss it may still lose data. Lucene cannot guarantee
* consistency on such devices. </p>
*
* <p> 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. </p>
*
* @see #prepareCommit
*
* @return The <a href="#sequence_number">sequence number</a>
* of the last operation in the commit. All sequence numbers &lt;= this value
* will be reflected in the commit, and all others will not.
*/
@Override
public final void commit() throws IOException {
public final long commit() throws IOException {
ensureOpen();
commitInternal(config.getMergePolicy());
return commitInternal(config.getMergePolicy());
}
/** Returns true if there may be changes that have not been
@ -3001,7 +3150,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
return changeCount.get() != lastCommitChangeCount || docWriter.anyChanges() || bufferedUpdatesStream.any();
}
private final void commitInternal(MergePolicy mergePolicy) throws IOException {
private final long commitInternal(MergePolicy mergePolicy) throws IOException {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "commit: start");
@ -3014,18 +3163,23 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
infoStream.message("IW", "commit: enter lock");
}
long seqNo;
if (pendingCommit == null) {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "commit: now prepare");
}
prepareCommitInternal(mergePolicy);
seqNo = prepareCommitInternal(mergePolicy);
} else {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "commit: already prepared");
}
seqNo = pendingSeqNo;
}
finishCommit();
return seqNo;
}
}
@ -3167,7 +3321,13 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
synchronized (fullFlushLock) {
boolean flushSuccess = false;
try {
anyChanges = docWriter.flushAllThreads();
long seqNo = docWriter.flushAllThreads();
if (seqNo < 0) {
seqNo = -seqNo;
anyChanges = true;
} else {
anyChanges = false;
}
if (!anyChanges) {
// flushCount is incremented in flushAllThreads
flushCount.incrementAndGet();
@ -4888,4 +5048,13 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
}
};
}
/** Returns the last <a href="#sequence_number">sequence number</a>, or 0
* if no index-changing operations have completed yet.
*
* @lucene.experimental */
public long getLastSequenceNumber() {
ensureOpen();
return docWriter.deleteQueue.getLastSequenceNumber();
}
}

View File

@ -421,7 +421,7 @@ public final class StandardDirectoryReader extends DirectoryReader {
@Override
public String toString() {
return "DirectoryReader.ReaderCommit(" + segmentsFileName + ")";
return "StandardDirectoryReader.ReaderCommit(" + segmentsFileName + " files=" + files + ")";
}
@Override

View File

@ -1,168 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.index;
import java.io.IOException;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.search.ControlledRealTimeReopenThread; // javadocs
import org.apache.lucene.search.Query;
import org.apache.lucene.store.Directory;
/** Class that tracks changes to a delegated
* IndexWriter, used by {@link
* ControlledRealTimeReopenThread} to ensure specific
* changes are visible. Create this class (passing your
* IndexWriter), and then pass this class to {@link
* ControlledRealTimeReopenThread}.
* Be sure to make all changes via the
* TrackingIndexWriter, otherwise {@link
* ControlledRealTimeReopenThread} won't know about the changes.
*
* @lucene.experimental */
public class TrackingIndexWriter {
private final IndexWriter writer;
private final AtomicLong indexingGen = new AtomicLong(1);
/** Create a {@code TrackingIndexWriter} wrapping the
* provided {@link IndexWriter}. */
public TrackingIndexWriter(IndexWriter writer) {
this.writer = writer;
}
/** Calls {@link
* IndexWriter#updateDocument(Term,Iterable)} and
* returns the generation that reflects this change. */
public long updateDocument(Term t, Iterable<? extends IndexableField> d) throws IOException {
writer.updateDocument(t, d);
// Return gen as of when indexing finished:
return indexingGen.get();
}
/** Calls {@link
* IndexWriter#updateDocuments(Term,Iterable)} and returns
* the generation that reflects this change. */
public long updateDocuments(Term t, Iterable<? extends Iterable<? extends IndexableField>> docs) throws IOException {
writer.updateDocuments(t, docs);
// Return gen as of when indexing finished:
return indexingGen.get();
}
/** Calls {@link IndexWriter#deleteDocuments(Term...)} and
* returns the generation that reflects this change. */
public long deleteDocuments(Term t) throws IOException {
writer.deleteDocuments(t);
// Return gen as of when indexing finished:
return indexingGen.get();
}
/** Calls {@link IndexWriter#deleteDocuments(Term...)} and
* returns the generation that reflects this change. */
public long deleteDocuments(Term... terms) throws IOException {
writer.deleteDocuments(terms);
// Return gen as of when indexing finished:
return indexingGen.get();
}
/** Calls {@link IndexWriter#deleteDocuments(Query...)} and
* returns the generation that reflects this change. */
public long deleteDocuments(Query q) throws IOException {
writer.deleteDocuments(q);
// Return gen as of when indexing finished:
return indexingGen.get();
}
/** Calls {@link IndexWriter#deleteDocuments(Query...)}
* and returns the generation that reflects this change. */
public long deleteDocuments(Query... queries) throws IOException {
writer.deleteDocuments(queries);
// Return gen as of when indexing finished:
return indexingGen.get();
}
/** Calls {@link IndexWriter#deleteAll} and returns the
* generation that reflects this change. */
public long deleteAll() throws IOException {
writer.deleteAll();
// Return gen as of when indexing finished:
return indexingGen.get();
}
/** Calls {@link IndexWriter#addDocument(Iterable)}
* and returns the generation that reflects this change. */
public long addDocument(Iterable<? extends IndexableField> d) throws IOException {
writer.addDocument(d);
// Return gen as of when indexing finished:
return indexingGen.get();
}
/** Calls {@link IndexWriter#addDocuments(Iterable)} and
* returns the generation that reflects this change. */
public long addDocuments(Iterable<? extends Iterable<? extends IndexableField>> docs) throws IOException {
writer.addDocuments(docs);
// Return gen as of when indexing finished:
return indexingGen.get();
}
/** Calls {@link IndexWriter#addIndexes(Directory...)} and
* returns the generation that reflects this change. */
public long addIndexes(Directory... dirs) throws IOException {
writer.addIndexes(dirs);
// Return gen as of when indexing finished:
return indexingGen.get();
}
/** Calls {@link IndexWriter#addIndexes(CodecReader...)}
* and returns the generation that reflects this change. */
public long addIndexes(CodecReader... readers) throws IOException {
writer.addIndexes(readers);
// Return gen as of when indexing finished:
return indexingGen.get();
}
/** Return the current generation being indexed. */
public long getGeneration() {
return indexingGen.get();
}
/** Return the wrapped {@link IndexWriter}. */
public IndexWriter getIndexWriter() {
return writer;
}
/** Return and increment current gen.
*
* @lucene.internal */
public long getAndIncrementGeneration() {
return indexingGen.getAndIncrement();
}
/** Cals {@link
* IndexWriter#tryDeleteDocument(IndexReader,int)} and
* returns the generation that reflects this change. */
public long tryDeleteDocument(IndexReader reader, int docID) throws IOException {
if (writer.tryDeleteDocument(reader, docID)) {
return indexingGen.get();
} else {
return -1;
}
}
}

View File

@ -34,7 +34,7 @@ public interface TwoPhaseCommit {
* 2-phase commit fails, {@link #rollback()} is called to discard all changes
* since last successful commit.
*/
public void prepareCommit() throws IOException;
public long prepareCommit() throws IOException;
/**
* The second phase of a 2-phase commit. Implementations should ideally do
@ -42,7 +42,7 @@ public interface TwoPhaseCommit {
* after it returns, the caller can assume that the changes were successfully
* committed to the underlying storage.
*/
public void commit() throws IOException;
public long commit() throws IOException;
/**
* Discards any changes that have occurred since the last commit. In a 2-phase
@ -51,5 +51,4 @@ public interface TwoPhaseCommit {
* back to their previous state.
*/
public void rollback() throws IOException;
}

View File

@ -275,13 +275,13 @@ final class BooleanScorer extends BulkScorer {
}
}
private void scoreWindowSingleScorer(BulkScorerAndDoc bulkScorer, LeafCollector collector,
private void scoreWindowSingleScorer(BulkScorerAndDoc bulkScorer, LeafCollector collector, LeafCollector singleClauseCollector,
Bits acceptDocs, int windowMin, int windowMax, int max) throws IOException {
assert tail.size() == 0;
final int nextWindowBase = head.top().next & ~MASK;
final int end = Math.max(windowMax, Math.min(max, nextWindowBase));
bulkScorer.score(collector, acceptDocs, windowMin, end);
bulkScorer.score(singleClauseCollector, acceptDocs, windowMin, end);
// reset the scorer that should be used for the general case
collector.setScorer(fakeScorer);
@ -304,7 +304,7 @@ final class BooleanScorer extends BulkScorer {
// special case: only one scorer can match in the current window,
// we can collect directly
final BulkScorerAndDoc bulkScorer = leads[0];
scoreWindowSingleScorer(bulkScorer, singleClauseCollector, acceptDocs, windowMin, windowMax, max);
scoreWindowSingleScorer(bulkScorer, collector, singleClauseCollector, acceptDocs, windowMin, windowMax, max);
return head.add(bulkScorer);
} else {
// general case, collect through a bit set first and then replay

View File

@ -23,16 +23,11 @@ import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.ReentrantLock;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.TrackingIndexWriter;
import org.apache.lucene.util.ThreadInterruptedException;
/** Utility class that runs a thread to manage periodicc
* reopens of a {@link ReferenceManager}, with methods to wait for a specific
* index changes to become visible. To use this class you
* must first wrap your {@link IndexWriter} with a {@link
* TrackingIndexWriter} and always use it to make changes
* to the index, saving the returned generation. Then,
* when a given search request needs to see a specific
* index changes to become visible. When a given search request needs to see a specific
* index change, call the {#waitForGeneration} to wait for
* that change to be visible. Note that this will only
* scale well if most searches do not need to wait for a
@ -44,7 +39,7 @@ public class ControlledRealTimeReopenThread<T> extends Thread implements Closeab
private final ReferenceManager<T> manager;
private final long targetMaxStaleNS;
private final long targetMinStaleNS;
private final TrackingIndexWriter writer;
private final IndexWriter writer;
private volatile boolean finish;
private volatile long waitingGen;
private volatile long searchingGen;
@ -69,7 +64,7 @@ public class ControlledRealTimeReopenThread<T> extends Thread implements Closeab
* is waiting for a specific generation to
* become visible.
*/
public ControlledRealTimeReopenThread(TrackingIndexWriter writer, ReferenceManager<T> manager, double targetMaxStaleSec, double targetMinStaleSec) {
public ControlledRealTimeReopenThread(IndexWriter writer, ReferenceManager<T> manager, double targetMaxStaleSec, double targetMinStaleSec) {
if (targetMaxStaleSec < targetMinStaleSec) {
throw new IllegalArgumentException("targetMaxScaleSec (= " + targetMaxStaleSec + ") < targetMinStaleSec (=" + targetMinStaleSec + ")");
}
@ -155,10 +150,7 @@ public class ControlledRealTimeReopenThread<T> extends Thread implements Closeab
* or false if maxMS wait time was exceeded
*/
public synchronized boolean waitForGeneration(long targetGen, int maxMS) throws InterruptedException {
final long curGen = writer.getGeneration();
if (targetGen > curGen) {
throw new IllegalArgumentException("targetGen=" + targetGen + " was never returned by the ReferenceManager instance (current gen=" + curGen + ")");
}
final long curGen = writer.getLastSequenceNumber();
if (targetGen > searchingGen) {
// Notify the reopen thread that the waitingGen has
// changed, so it may wake up and realize it should
@ -240,7 +232,7 @@ public class ControlledRealTimeReopenThread<T> extends Thread implements Closeab
// Save the gen as of when we started the reopen; the
// listener (HandleRefresh above) copies this to
// searchingGen once the reopen completes:
refreshStartGen = writer.getAndIncrementGeneration();
refreshStartGen = writer.getLastSequenceNumber();
try {
manager.maybeRefreshBlocking();
} catch (IOException ioe) {

View File

@ -45,7 +45,12 @@ public abstract class FilterLeafCollector implements LeafCollector {
@Override
public String toString() {
return getClass().getSimpleName() + "(" + in + ")";
String name = getClass().getSimpleName();
if (name.length() == 0) {
// an anonoymous subclass will have empty name?
name = "FilterLeafCollector";
}
return name + "(" + in + ")";
}
}

View File

@ -149,6 +149,11 @@ public abstract class IndexInput extends DataInput implements Cloneable,Closeabl
slice.seek(pos);
return slice.readLong();
}
@Override
public String toString() {
return "RandomAccessInput(" + IndexInput.this.toString() + ")";
}
};
}
}

View File

@ -43,8 +43,8 @@ public class TestDocumentsWriterDeleteQueue extends LuceneTestCase {
}
DeleteSlice slice1 = queue.newSlice();
DeleteSlice slice2 = queue.newSlice();
BufferedUpdates bd1 = new BufferedUpdates();
BufferedUpdates bd2 = new BufferedUpdates();
BufferedUpdates bd1 = new BufferedUpdates("bd1");
BufferedUpdates bd2 = new BufferedUpdates("bd2");
int last1 = 0;
int last2 = 0;
Set<Term> uniqueValues = new HashSet<>();
@ -225,7 +225,7 @@ public class TestDocumentsWriterDeleteQueue extends LuceneTestCase {
this.index = index;
this.ids = ids;
this.slice = queue.newSlice();
deletes = new BufferedUpdates();
deletes = new BufferedUpdates("deletes");
this.latch = latch;
}

View File

@ -93,7 +93,6 @@ public class TestIndexWriterConfig extends LuceneTestCase {
getters.add("getIndexingChain");
getters.add("getMergedSegmentWarmer");
getters.add("getMergePolicy");
getters.add("getMaxThreadStates");
getters.add("getReaderPooling");
getters.add("getIndexerThreadPool");
getters.add("getFlushPolicy");

View File

@ -1238,8 +1238,8 @@ public class TestIndexWriterDelete extends LuceneTestCase {
iwc.setOpenMode(IndexWriterConfig.OpenMode.APPEND);
w = new IndexWriter(d, iwc);
IndexReader r = DirectoryReader.open(w, false, false);
assertTrue(w.tryDeleteDocument(r, 1));
assertTrue(w.tryDeleteDocument(r.leaves().get(0).reader(), 0));
assertTrue(w.tryDeleteDocument(r, 1) != -1);
assertTrue(w.tryDeleteDocument(r.leaves().get(0).reader(), 0) != -1);
r.close();
w.close();

View File

@ -0,0 +1,640 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.index;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.NumericDocValuesField;
import org.apache.lucene.document.StoredField;
import org.apache.lucene.document.StringField;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.search.TopDocs;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
public class TestIndexingSequenceNumbers extends LuceneTestCase {
public void testBasic() throws Exception {
Directory dir = newDirectory();
IndexWriter w = new IndexWriter(dir, newIndexWriterConfig());
long a = w.addDocument(new Document());
long b = w.addDocument(new Document());
assertTrue(b > a);
w.close();
dir.close();
}
public void testAfterRefresh() throws Exception {
Directory dir = newDirectory();
IndexWriter w = new IndexWriter(dir, newIndexWriterConfig());
long a = w.addDocument(new Document());
DirectoryReader.open(w).close();
long b = w.addDocument(new Document());
assertTrue(b > a);
w.close();
dir.close();
}
public void testAfterCommit() throws Exception {
Directory dir = newDirectory();
IndexWriter w = new IndexWriter(dir, newIndexWriterConfig());
long a = w.addDocument(new Document());
w.commit();
long b = w.addDocument(new Document());
assertTrue(b > a);
w.close();
dir.close();
}
public void testStressUpdateSameID() throws Exception {
int iters = atLeast(100);
for(int iter=0;iter<iters;iter++) {
Directory dir = newDirectory();
final RandomIndexWriter w = new RandomIndexWriter(random(), dir);
Thread[] threads = new Thread[TestUtil.nextInt(random(), 2, 5)];
final CountDownLatch startingGun = new CountDownLatch(1);
final long[] seqNos = new long[threads.length];
final Term id = new Term("id", "id");
// multiple threads update the same document
for(int i=0;i<threads.length;i++) {
final int threadID = i;
threads[i] = new Thread() {
@Override
public void run() {
try {
Document doc = new Document();
doc.add(new StoredField("thread", threadID));
doc.add(new StringField("id", "id", Field.Store.NO));
startingGun.await();
for(int j=0;j<100;j++) {
if (random().nextBoolean()) {
seqNos[threadID] = w.updateDocument(id, doc);
} else {
List<Document> docs = new ArrayList<>();
docs.add(doc);
seqNos[threadID] = w.updateDocuments(id, docs);
}
}
} catch (Exception e) {
throw new RuntimeException(e);
}
}
};
threads[i].start();
}
startingGun.countDown();
for(Thread thread : threads) {
thread.join();
}
// now confirm that the reported sequence numbers agree with the index:
int maxThread = 0;
Set<Long> allSeqNos = new HashSet<>();
for(int i=0;i<threads.length;i++) {
allSeqNos.add(seqNos[i]);
if (seqNos[i] > seqNos[maxThread]) {
maxThread = i;
}
}
// make sure all sequence numbers were different
assertEquals(threads.length, allSeqNos.size());
DirectoryReader r = w.getReader();
IndexSearcher s = newSearcher(r);
TopDocs hits = s.search(new TermQuery(id), 1);
assertEquals(1, hits.totalHits);
Document doc = r.document(hits.scoreDocs[0].doc);
assertEquals(maxThread, doc.getField("thread").numericValue().intValue());
r.close();
w.close();
dir.close();
}
}
static class Operation {
// 0 = update, 1 = delete, 2 = commit, 3 = add
byte what;
int id;
int threadID;
long seqNo;
}
public void testStressConcurrentCommit() throws Exception {
final int opCount = atLeast(10000);
final int idCount = TestUtil.nextInt(random(), 10, 1000);
Directory dir = newDirectory();
IndexWriterConfig iwc = newIndexWriterConfig();
iwc.setIndexDeletionPolicy(NoDeletionPolicy.INSTANCE);
// Cannot use RIW since it randomly commits:
final IndexWriter w = new IndexWriter(dir, iwc);
final int numThreads = TestUtil.nextInt(random(), 2, 10);
Thread[] threads = new Thread[numThreads];
//System.out.println("TEST: iter=" + iter + " opCount=" + opCount + " idCount=" + idCount + " threadCount=" + threads.length);
final CountDownLatch startingGun = new CountDownLatch(1);
List<List<Operation>> threadOps = new ArrayList<>();
Object commitLock = new Object();
final List<Operation> commits = new ArrayList<>();
// multiple threads update the same set of documents, and we randomly commit, recording the commit seqNo and then opening each commit in
// the end to verify it reflects the correct updates
for(int i=0;i<threads.length;i++) {
final List<Operation> ops = new ArrayList<>();
threadOps.add(ops);
final int threadID = i;
threads[i] = new Thread() {
@Override
public void run() {
try {
startingGun.await();
for(int i=0;i<opCount;i++) {
Operation op = new Operation();
op.threadID = threadID;
if (random().nextInt(500) == 17) {
op.what = 2;
synchronized(commitLock) {
op.seqNo = w.commit();
if (op.seqNo != -1) {
commits.add(op);
}
}
} else {
op.id = random().nextInt(idCount);
Term idTerm = new Term("id", "" + op.id);
if (random().nextInt(10) == 1) {
op.what = 1;
if (random().nextBoolean()) {
op.seqNo = w.deleteDocuments(idTerm);
} else {
op.seqNo = w.deleteDocuments(new TermQuery(idTerm));
}
} else {
Document doc = new Document();
doc.add(new StoredField("thread", threadID));
doc.add(new StringField("id", "" + op.id, Field.Store.NO));
if (random().nextBoolean()) {
List<Document> docs = new ArrayList<>();
docs.add(doc);
op.seqNo = w.updateDocuments(idTerm, docs);
} else {
op.seqNo = w.updateDocument(idTerm, doc);
}
op.what = 0;
}
ops.add(op);
}
}
} catch (Exception e) {
throw new RuntimeException(e);
}
}
};
threads[i].start();
}
startingGun.countDown();
for(Thread thread : threads) {
thread.join();
}
Operation commitOp = new Operation();
commitOp.seqNo = w.commit();
if (commitOp.seqNo != -1) {
commits.add(commitOp);
}
List<IndexCommit> indexCommits = DirectoryReader.listCommits(dir);
assertEquals(commits.size(), indexCommits.size());
int[] expectedThreadIDs = new int[idCount];
long[] seqNos = new long[idCount];
//System.out.println("TEST: " + commits.size() + " commits");
for(int i=0;i<commits.size();i++) {
// this commit point should reflect all operations <= this seqNo
long commitSeqNo = commits.get(i).seqNo;
//System.out.println(" commit " + i + ": seqNo=" + commitSeqNo + " segs=" + indexCommits.get(i));
Arrays.fill(expectedThreadIDs, -1);
Arrays.fill(seqNos, 0);
for(int threadID=0;threadID<threadOps.size();threadID++) {
long lastSeqNo = 0;
for(Operation op : threadOps.get(threadID)) {
if (op.seqNo <= commitSeqNo && op.seqNo > seqNos[op.id]) {
seqNos[op.id] = op.seqNo;
if (op.what == 0) {
expectedThreadIDs[op.id] = threadID;
} else {
expectedThreadIDs[op.id] = -1;
}
}
assertTrue(op.seqNo > lastSeqNo);
lastSeqNo = op.seqNo;
}
}
DirectoryReader r = DirectoryReader.open(indexCommits.get(i));
IndexSearcher s = new IndexSearcher(r);
for(int id=0;id<idCount;id++) {
//System.out.println("TEST: check id=" + id + " expectedThreadID=" + expectedThreadIDs[id]);
TopDocs hits = s.search(new TermQuery(new Term("id", ""+id)), 1);
if (expectedThreadIDs[id] != -1) {
assertEquals(1, hits.totalHits);
Document doc = r.document(hits.scoreDocs[0].doc);
int actualThreadID = doc.getField("thread").numericValue().intValue();
if (expectedThreadIDs[id] != actualThreadID) {
System.out.println("FAIL: id=" + id + " expectedThreadID=" + expectedThreadIDs[id] + " vs actualThreadID=" + actualThreadID + " commitSeqNo=" + commitSeqNo + " numThreads=" + numThreads);
for(int threadID=0;threadID<threadOps.size();threadID++) {
for(Operation op : threadOps.get(threadID)) {
if (id == op.id) {
System.out.println(" threadID=" + threadID + " seqNo=" + op.seqNo + " " + (op.what == 2 ? "updated" : "deleted"));
}
}
}
assertEquals("id=" + id, expectedThreadIDs[id], actualThreadID);
}
} else if (hits.totalHits != 0) {
System.out.println("FAIL: id=" + id + " expectedThreadID=" + expectedThreadIDs[id] + " vs totalHits=" + hits.totalHits + " commitSeqNo=" + commitSeqNo + " numThreads=" + numThreads);
for(int threadID=0;threadID<threadOps.size();threadID++) {
for(Operation op : threadOps.get(threadID)) {
if (id == op.id) {
System.out.println(" threadID=" + threadID + " seqNo=" + op.seqNo + " " + (op.what == 2 ? "updated" : "del"));
}
}
}
assertEquals(0, hits.totalHits);
}
}
w.close();
r.close();
}
dir.close();
}
public void testStressConcurrentDocValuesUpdatesCommit() throws Exception {
final int opCount = atLeast(10000);
final int idCount = TestUtil.nextInt(random(), 10, 1000);
Directory dir = newDirectory();
IndexWriterConfig iwc = newIndexWriterConfig();
iwc.setIndexDeletionPolicy(NoDeletionPolicy.INSTANCE);
// Cannot use RIW since it randomly commits:
final IndexWriter w = new IndexWriter(dir, iwc);
final int numThreads = TestUtil.nextInt(random(), 2, 10);
if (VERBOSE) {
System.out.println("TEST: numThreads=" + numThreads);
}
Thread[] threads = new Thread[numThreads];
//System.out.println("TEST: iter=" + iter + " opCount=" + opCount + " idCount=" + idCount + " threadCount=" + threads.length);
final CountDownLatch startingGun = new CountDownLatch(1);
List<List<Operation>> threadOps = new ArrayList<>();
Object commitLock = new Object();
final List<Operation> commits = new ArrayList<>();
List<Operation> ops1 = new ArrayList<>();
threadOps.add(ops1);
// pre-index every ID so none are missing:
for(int id=0;id<idCount;id++) {
int threadID = 0;
Operation op = new Operation();
op.threadID = threadID;
op.id = id;
Document doc = new Document();
doc.add(new StoredField("thread", threadID));
doc.add(new NumericDocValuesField("thread", threadID));
doc.add(new StringField("id", "" + id, Field.Store.NO));
op.seqNo = w.addDocument(doc);
ops1.add(op);
}
// multiple threads update the same set of documents, and we randomly commit, recording the commit seqNo and then opening each commit in
// the end to verify it reflects the correct updates
for(int i=0;i<threads.length;i++) {
final List<Operation> ops;
if (i == 0) {
ops = threadOps.get(0);
} else {
ops = new ArrayList<>();
threadOps.add(ops);
}
final int threadID = i;
threads[i] = new Thread() {
@Override
public void run() {
try {
startingGun.await();
for(int i=0;i<opCount;i++) {
Operation op = new Operation();
op.threadID = threadID;
if (random().nextInt(500) == 17) {
op.what = 2;
synchronized(commitLock) {
op.seqNo = w.commit();
if (op.seqNo != -1) {
commits.add(op);
}
}
} else {
op.id = random().nextInt(idCount);
Term idTerm = new Term("id", "" + op.id);
op.seqNo = w.updateNumericDocValue(idTerm, "thread", threadID);
op.what = 0;
ops.add(op);
}
}
} catch (Exception e) {
throw new RuntimeException(e);
}
}
};
threads[i].setName("thread" + i);
threads[i].start();
}
startingGun.countDown();
for(Thread thread : threads) {
thread.join();
}
Operation commitOp = new Operation();
commitOp.seqNo = w.commit();
if (commitOp.seqNo != -1) {
commits.add(commitOp);
}
List<IndexCommit> indexCommits = DirectoryReader.listCommits(dir);
assertEquals(commits.size(), indexCommits.size());
int[] expectedThreadIDs = new int[idCount];
long[] seqNos = new long[idCount];
//System.out.println("TEST: " + commits.size() + " commits");
for(int i=0;i<commits.size();i++) {
// this commit point should reflect all operations <= this seqNo
long commitSeqNo = commits.get(i).seqNo;
//System.out.println(" commit " + i + ": seqNo=" + commitSeqNo + " segs=" + indexCommits.get(i));
Arrays.fill(expectedThreadIDs, -1);
Arrays.fill(seqNos, 0);
for(int threadID=0;threadID<threadOps.size();threadID++) {
long lastSeqNo = 0;
for(Operation op : threadOps.get(threadID)) {
if (op.seqNo <= commitSeqNo && op.seqNo > seqNos[op.id]) {
seqNos[op.id] = op.seqNo;
assert op.what == 0;
expectedThreadIDs[op.id] = threadID;
}
assertTrue(op.seqNo > lastSeqNo);
lastSeqNo = op.seqNo;
}
}
DirectoryReader r = DirectoryReader.open(indexCommits.get(i));
IndexSearcher s = new IndexSearcher(r);
NumericDocValues docValues = MultiDocValues.getNumericValues(r, "thread");
for(int id=0;id<idCount;id++) {
//System.out.println("TEST: check id=" + id + " expectedThreadID=" + expectedThreadIDs[id]);
TopDocs hits = s.search(new TermQuery(new Term("id", ""+id)), 1);
// We pre-add all ids up front:
assert expectedThreadIDs[id] != -1;
assertEquals(1, hits.totalHits);
int actualThreadID = (int) docValues.get(hits.scoreDocs[0].doc);
if (expectedThreadIDs[id] != actualThreadID) {
System.out.println("FAIL: commit=" + i + " (of " + commits.size() + ") id=" + id + " expectedThreadID=" + expectedThreadIDs[id] + " vs actualThreadID=" + actualThreadID + " commitSeqNo=" + commitSeqNo + " numThreads=" + numThreads + " reader=" + r + " commit=" + indexCommits.get(i));
for(int threadID=0;threadID<threadOps.size();threadID++) {
for(Operation op : threadOps.get(threadID)) {
if (id == op.id) {
System.out.println(" threadID=" + threadID + " seqNo=" + op.seqNo);
}
}
}
assertEquals("id=" + id + " docID=" + hits.scoreDocs[0].doc, expectedThreadIDs[id], actualThreadID);
}
}
w.close();
r.close();
}
dir.close();
}
public void testStressConcurrentAddAndDeleteAndCommit() throws Exception {
final int opCount = atLeast(10000);
final int idCount = TestUtil.nextInt(random(), 10, 1000);
Directory dir = newDirectory();
IndexWriterConfig iwc = newIndexWriterConfig();
iwc.setIndexDeletionPolicy(NoDeletionPolicy.INSTANCE);
// Cannot use RIW since it randomly commits:
final IndexWriter w = new IndexWriter(dir, iwc);
final int numThreads = TestUtil.nextInt(random(), 2, 5);
Thread[] threads = new Thread[numThreads];
//System.out.println("TEST: iter=" + iter + " opCount=" + opCount + " idCount=" + idCount + " threadCount=" + threads.length);
final CountDownLatch startingGun = new CountDownLatch(1);
List<List<Operation>> threadOps = new ArrayList<>();
Object commitLock = new Object();
final List<Operation> commits = new ArrayList<>();
// multiple threads update the same set of documents, and we randomly commit
for(int i=0;i<threads.length;i++) {
final List<Operation> ops = new ArrayList<>();
threadOps.add(ops);
final int threadID = i;
threads[i] = new Thread() {
@Override
public void run() {
try {
startingGun.await();
for(int i=0;i<opCount;i++) {
Operation op = new Operation();
op.threadID = threadID;
if (random().nextInt(500) == 17) {
op.what = 2;
synchronized(commitLock) {
op.seqNo = w.commit();
if (op.seqNo != -1) {
commits.add(op);
}
}
} else {
op.id = random().nextInt(idCount);
Term idTerm = new Term("id", "" + op.id);
if (random().nextInt(10) == 1) {
op.what = 1;
if (random().nextBoolean()) {
op.seqNo = w.deleteDocuments(idTerm);
} else {
op.seqNo = w.deleteDocuments(new TermQuery(idTerm));
}
} else {
Document doc = new Document();
doc.add(new StoredField("threadop", threadID + "-" + ops.size()));
doc.add(new StringField("id", "" + op.id, Field.Store.NO));
if (random().nextBoolean()) {
List<Document> docs = new ArrayList<>();
docs.add(doc);
op.seqNo = w.addDocuments(docs);
} else {
op.seqNo = w.addDocument(doc);
}
op.what = 3;
}
ops.add(op);
}
}
} catch (Exception e) {
throw new RuntimeException(e);
}
}
};
threads[i].setName("thread" + threadID);
threads[i].start();
}
startingGun.countDown();
for(Thread thread : threads) {
thread.join();
}
Operation commitOp = new Operation();
commitOp.seqNo = w.commit();
if (commitOp.seqNo != -1) {
commits.add(commitOp);
}
List<IndexCommit> indexCommits = DirectoryReader.listCommits(dir);
assertEquals(commits.size(), indexCommits.size());
// how many docs with this id are expected:
int[] expectedCounts = new int[idCount];
long[] lastDelSeqNos = new long[idCount];
//System.out.println("TEST: " + commits.size() + " commits");
for(int i=0;i<commits.size();i++) {
// this commit point should reflect all operations <= this seqNo
long commitSeqNo = commits.get(i).seqNo;
//System.out.println(" commit " + i + ": seqNo=" + commitSeqNo + " segs=" + indexCommits.get(i));
// first find the highest seqNo of the last delete op, for each id, prior to this commit:
Arrays.fill(lastDelSeqNos, -1);
for(int threadID=0;threadID<threadOps.size();threadID++) {
long lastSeqNo = 0;
for(Operation op : threadOps.get(threadID)) {
if (op.what == 1 && op.seqNo <= commitSeqNo && op.seqNo > lastDelSeqNos[op.id]) {
lastDelSeqNos[op.id] = op.seqNo;
}
// within one thread the seqNos must only increase:
assertTrue(op.seqNo > lastSeqNo);
lastSeqNo = op.seqNo;
}
}
// then count how many adds happened since the last delete and before this commit:
Arrays.fill(expectedCounts, 0);
for(int threadID=0;threadID<threadOps.size();threadID++) {
for(Operation op : threadOps.get(threadID)) {
if (op.what == 3 && op.seqNo <= commitSeqNo && op.seqNo > lastDelSeqNos[op.id]) {
expectedCounts[op.id]++;
}
}
}
DirectoryReader r = DirectoryReader.open(indexCommits.get(i));
IndexSearcher s = new IndexSearcher(r);
for(int id=0;id<idCount;id++) {
//System.out.println("TEST: check id=" + id + " expectedThreadID=" + expectedThreadIDs[id]);
int actualCount = s.count(new TermQuery(new Term("id", ""+id)));
if (expectedCounts[id] != actualCount) {
System.out.println("TEST: FAIL r=" + r + " id=" + id + " commitSeqNo=" + commitSeqNo);
for(int threadID=0;threadID<threadOps.size();threadID++) {
int opCount2 = 0;
for(Operation op : threadOps.get(threadID)) {
if (op.id == id) {
boolean shouldCount = op.seqNo <= commitSeqNo && op.seqNo > lastDelSeqNos[op.id];
System.out.println(" id=" + id + " what=" + op.what + " threadop=" + threadID + "-" + opCount2 + " seqNo=" + op.seqNo + " vs lastDelSeqNo=" + lastDelSeqNos[op.id] + " shouldCount=" + shouldCount);
}
opCount2++;
}
}
TopDocs hits = s.search(new TermQuery(new Term("id", ""+id)), 1+actualCount);
for(ScoreDoc hit : hits.scoreDocs) {
System.out.println(" hit: " + s.doc(hit.doc).get("threadop"));
}
for(LeafReaderContext ctx : r.leaves()) {
System.out.println(" sub=" + ctx.reader());
Bits liveDocs = ctx.reader().getLiveDocs();
for(int docID=0;docID<ctx.reader().maxDoc();docID++) {
System.out.println(" docID=" + docID + " threadop=" + ctx.reader().document(docID).get("threadop") + (liveDocs != null && liveDocs.get(docID) == false ? " (deleted)" : ""));
}
}
assertEquals("commit " + i + " of " + commits.size() + " id=" + id + " reader=" + r, expectedCounts[id], actualCount);
}
}
w.close();
r.close();
}
dir.close();
}
public void testDeleteAll() throws Exception {
Directory dir = newDirectory();
IndexWriter w = new IndexWriter(dir, newIndexWriterConfig());
long a = w.addDocument(new Document());
long b = w.deleteAll();
assertTrue(a < b);
long c = w.commit();
assertTrue(b < c);
w.close();
dir.close();
}
}

View File

@ -80,7 +80,7 @@ public class TestRollingUpdates extends LuceneTestCase {
if (s != null && updateCount < SIZE) {
TopDocs hits = s.search(new TermQuery(idTerm), 1);
assertEquals(1, hits.totalHits);
doUpdate = !w.tryDeleteDocument(r, hits.scoreDocs[0].doc);
doUpdate = w.tryDeleteDocument(r, hits.scoreDocs[0].doc) == -1;
if (VERBOSE) {
if (doUpdate) {
System.out.println(" tryDeleteDocument failed");

View File

@ -79,8 +79,6 @@ public class TestTryDelete extends LuceneTestCase
ReferenceManager<IndexSearcher> mgr = new SearcherManager(writer,
new SearcherFactory());
TrackingIndexWriter mgrWriter = new TrackingIndexWriter(writer);
IndexSearcher searcher = mgr.acquire();
TopDocs topDocs = searcher.search(new TermQuery(new Term("foo", "0")),
@ -90,10 +88,10 @@ public class TestTryDelete extends LuceneTestCase
long result;
if (random().nextBoolean()) {
IndexReader r = DirectoryReader.open(writer);
result = mgrWriter.tryDeleteDocument(r, 0);
result = writer.tryDeleteDocument(r, 0);
r.close();
} else {
result = mgrWriter.tryDeleteDocument(searcher.getIndexReader(), 0);
result = writer.tryDeleteDocument(searcher.getIndexReader(), 0);
}
// The tryDeleteDocument should have succeeded:
@ -132,10 +130,9 @@ public class TestTryDelete extends LuceneTestCase
100);
assertEquals(1, topDocs.totalHits);
TrackingIndexWriter mgrWriter = new TrackingIndexWriter(writer);
long result = mgrWriter.tryDeleteDocument(DirectoryReader.open(writer), 0);
long result = writer.tryDeleteDocument(DirectoryReader.open(writer), 0);
assertEquals(1, result);
assertTrue(result != -1);
writer.commit();
@ -175,11 +172,9 @@ public class TestTryDelete extends LuceneTestCase
100);
assertEquals(1, topDocs.totalHits);
TrackingIndexWriter mgrWriter = new TrackingIndexWriter(writer);
long result = mgrWriter.deleteDocuments(new TermQuery(new Term("foo",
"0")));
long result = writer.deleteDocuments(new TermQuery(new Term("foo", "0")));
assertEquals(1, result);
assertTrue(result != -1);
// writer.commit();

View File

@ -40,29 +40,31 @@ public class TestTwoPhaseCommitTool extends LuceneTestCase {
}
@Override
public void prepareCommit() throws IOException {
prepareCommit(null);
public long prepareCommit() throws IOException {
return prepareCommit(null);
}
public void prepareCommit(Map<String, String> commitData) throws IOException {
public long prepareCommit(Map<String, String> commitData) throws IOException {
this.prepareCommitData = commitData;
assertFalse("commit should not have been called before all prepareCommit were", commitCalled);
if (failOnPrepare) {
throw new IOException("failOnPrepare");
}
return 1;
}
@Override
public void commit() throws IOException {
commit(null);
public long commit() throws IOException {
return commit(null);
}
public void commit(Map<String, String> commitData) throws IOException {
public long commit(Map<String, String> commitData) throws IOException {
this.commitData = commitData;
commitCalled = true;
if (failOnCommit) {
throw new RuntimeException("failOnCommit");
}
return 1;
}
@Override

View File

@ -18,15 +18,18 @@ package org.apache.lucene.search;
import java.util.Arrays;
import java.util.Random;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.search.similarities.ClassicSimilarity;
import org.apache.lucene.search.similarities.Similarity;
import org.apache.lucene.store.Directory;
@ -42,24 +45,45 @@ import org.junit.Test;
*/
public class TestBoolean2 extends LuceneTestCase {
private static IndexSearcher searcher;
private static IndexSearcher singleSegmentSearcher;
private static IndexSearcher bigSearcher;
private static IndexReader reader;
private static IndexReader littleReader;
private static int NUM_EXTRA_DOCS = 6000;
private static IndexReader singleSegmentReader;
/** num of empty docs injected between every doc in the (main) index */
private static int NUM_FILLER_DOCS;
/** num of empty docs injected prior to the first doc in the (main) index */
private static int PRE_FILLER_DOCS;
/** num "extra" docs containing value in "field2" added to the "big" clone of the index */
private static final int NUM_EXTRA_DOCS = 6000;
public static final String field = "field";
private static Directory directory;
private static Directory singleSegmentDirectory;
private static Directory dir2;
private static int mulFactor;
@BeforeClass
public static void beforeClass() throws Exception {
// in some runs, test immediate adjacency of matches - in others, force a full bucket gap betwen docs
NUM_FILLER_DOCS = random().nextBoolean() ? 0 : BooleanScorer.SIZE;
PRE_FILLER_DOCS = TestUtil.nextInt(random(), 0, (NUM_FILLER_DOCS / 2));
directory = newDirectory();
RandomIndexWriter writer= new RandomIndexWriter(random(), directory, newIndexWriterConfig(new MockAnalyzer(random())).setMergePolicy(newLogMergePolicy()));
Document doc = new Document();
for (int filler = 0; filler < PRE_FILLER_DOCS; filler++) {
writer.addDocument(doc);
}
for (int i = 0; i < docFields.length; i++) {
Document doc = new Document();
doc.add(newTextField(field, docFields[i], Field.Store.NO));
writer.addDocument(doc);
doc = new Document();
for (int filler = 0; filler < NUM_FILLER_DOCS; filler++) {
writer.addDocument(doc);
}
}
writer.close();
littleReader = DirectoryReader.open(directory);
@ -67,6 +91,18 @@ public class TestBoolean2 extends LuceneTestCase {
// this is intentionally using the baseline sim, because it compares against bigSearcher (which uses a random one)
searcher.setSimilarity(new ClassicSimilarity());
// make a copy of our index using a single segment
singleSegmentDirectory = new MockDirectoryWrapper(random(), TestUtil.ramCopyOf(directory));
IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random()));
// we need docID order to be preserved:
iwc.setMergePolicy(newLogMergePolicy());
try (IndexWriter w = new IndexWriter(singleSegmentDirectory, iwc)) {
w.forceMerge(1, true);
}
singleSegmentReader = DirectoryReader.open(singleSegmentDirectory);
singleSegmentSearcher = newSearcher(singleSegmentReader);
singleSegmentSearcher.setSimilarity(searcher.getSimilarity(true));
// Make big index
dir2 = new MockDirectoryWrapper(random(), TestUtil.ramCopyOf(directory));
@ -86,12 +122,12 @@ public class TestBoolean2 extends LuceneTestCase {
docCount = w.maxDoc();
w.close();
mulFactor *= 2;
} while(docCount < 3000);
} while(docCount < 3000 * NUM_FILLER_DOCS);
RandomIndexWriter w = new RandomIndexWriter(random(), dir2,
newIndexWriterConfig(new MockAnalyzer(random()))
.setMaxBufferedDocs(TestUtil.nextInt(random(), 50, 1000)));
Document doc = new Document();
doc = new Document();
doc.add(newTextField("field2", "xxx", Field.Store.NO));
for(int i=0;i<NUM_EXTRA_DOCS/2;i++) {
w.addDocument(doc);
@ -110,8 +146,13 @@ public class TestBoolean2 extends LuceneTestCase {
public static void afterClass() throws Exception {
reader.close();
littleReader.close();
singleSegmentReader.close();
dir2.close();
directory.close();
singleSegmentDirectory.close();
singleSegmentSearcher = null;
singleSegmentReader = null;
singleSegmentDirectory = null;
searcher = null;
reader = null;
littleReader = null;
@ -128,26 +169,57 @@ public class TestBoolean2 extends LuceneTestCase {
};
public void queriesTest(Query query, int[] expDocNrs) throws Exception {
// adjust the expected doc numbers according to our filler docs
if (0 < NUM_FILLER_DOCS) {
expDocNrs = Arrays.copyOf(expDocNrs, expDocNrs.length);
for (int i=0; i < expDocNrs.length; i++) {
expDocNrs[i] = PRE_FILLER_DOCS + ((NUM_FILLER_DOCS + 1) * expDocNrs[i]);
}
}
final int topDocsToCheck = atLeast(1000);
// The asserting searcher will sometimes return the bulk scorer and
// sometimes return a default impl around the scorer so that we can
// compare BS1 and BS2
TopScoreDocCollector collector = TopScoreDocCollector.create(1000);
TopScoreDocCollector collector = TopScoreDocCollector.create(topDocsToCheck);
searcher.search(query, collector);
ScoreDoc[] hits1 = collector.topDocs().scoreDocs;
collector = TopScoreDocCollector.create(1000);
collector = TopScoreDocCollector.create(topDocsToCheck);
searcher.search(query, collector);
ScoreDoc[] hits2 = collector.topDocs().scoreDocs;
CheckHits.checkHitsQuery(query, hits1, hits2, expDocNrs);
// Since we have no deleted docs, we should also be able to verify identical matches &
// scores against an single segment copy of our index
collector = TopScoreDocCollector.create(topDocsToCheck);
singleSegmentSearcher.search(query, collector);
hits2 = collector.topDocs().scoreDocs;
CheckHits.checkHitsQuery(query, hits1, hits2, expDocNrs);
// sanity check expected num matches in bigSearcher
assertEquals(mulFactor * collector.totalHits,
bigSearcher.search(query, 1).totalHits);
// now check 2 diff scorers from the bigSearcher as well
collector = TopScoreDocCollector.create(topDocsToCheck);
bigSearcher.search(query, collector);
hits1 = collector.topDocs().scoreDocs;
collector = TopScoreDocCollector.create(topDocsToCheck);
bigSearcher.search(query, collector);
hits2 = collector.topDocs().scoreDocs;
// NOTE: just comparing results, not vetting against expDocNrs
// since we have dups in bigSearcher
CheckHits.checkEqual(query, hits1, hits2);
CheckHits.checkHitsQuery(query, hits1, hits2, expDocNrs);
}
@Test
public void testQueries01() throws Exception {
BooleanQuery.Builder query = new BooleanQuery.Builder();
query.setDisableCoord(random().nextBoolean());
query.add(new TermQuery(new Term(field, "w3")), BooleanClause.Occur.MUST);
query.add(new TermQuery(new Term(field, "xx")), BooleanClause.Occur.MUST);
int[] expDocNrs = {2,3};
@ -157,6 +229,7 @@ public class TestBoolean2 extends LuceneTestCase {
@Test
public void testQueries02() throws Exception {
BooleanQuery.Builder query = new BooleanQuery.Builder();
query.setDisableCoord(random().nextBoolean());
query.add(new TermQuery(new Term(field, "w3")), BooleanClause.Occur.MUST);
query.add(new TermQuery(new Term(field, "xx")), BooleanClause.Occur.SHOULD);
int[] expDocNrs = {2,3,1,0};
@ -166,6 +239,7 @@ public class TestBoolean2 extends LuceneTestCase {
@Test
public void testQueries03() throws Exception {
BooleanQuery.Builder query = new BooleanQuery.Builder();
query.setDisableCoord(random().nextBoolean());
query.add(new TermQuery(new Term(field, "w3")), BooleanClause.Occur.SHOULD);
query.add(new TermQuery(new Term(field, "xx")), BooleanClause.Occur.SHOULD);
int[] expDocNrs = {2,3,1,0};
@ -175,6 +249,7 @@ public class TestBoolean2 extends LuceneTestCase {
@Test
public void testQueries04() throws Exception {
BooleanQuery.Builder query = new BooleanQuery.Builder();
query.setDisableCoord(random().nextBoolean());
query.add(new TermQuery(new Term(field, "w3")), BooleanClause.Occur.SHOULD);
query.add(new TermQuery(new Term(field, "xx")), BooleanClause.Occur.MUST_NOT);
int[] expDocNrs = {1,0};
@ -184,6 +259,7 @@ public class TestBoolean2 extends LuceneTestCase {
@Test
public void testQueries05() throws Exception {
BooleanQuery.Builder query = new BooleanQuery.Builder();
query.setDisableCoord(random().nextBoolean());
query.add(new TermQuery(new Term(field, "w3")), BooleanClause.Occur.MUST);
query.add(new TermQuery(new Term(field, "xx")), BooleanClause.Occur.MUST_NOT);
int[] expDocNrs = {1,0};
@ -193,6 +269,7 @@ public class TestBoolean2 extends LuceneTestCase {
@Test
public void testQueries06() throws Exception {
BooleanQuery.Builder query = new BooleanQuery.Builder();
query.setDisableCoord(random().nextBoolean());
query.add(new TermQuery(new Term(field, "w3")), BooleanClause.Occur.MUST);
query.add(new TermQuery(new Term(field, "xx")), BooleanClause.Occur.MUST_NOT);
query.add(new TermQuery(new Term(field, "w5")), BooleanClause.Occur.MUST_NOT);
@ -203,6 +280,7 @@ public class TestBoolean2 extends LuceneTestCase {
@Test
public void testQueries07() throws Exception {
BooleanQuery.Builder query = new BooleanQuery.Builder();
query.setDisableCoord(random().nextBoolean());
query.add(new TermQuery(new Term(field, "w3")), BooleanClause.Occur.MUST_NOT);
query.add(new TermQuery(new Term(field, "xx")), BooleanClause.Occur.MUST_NOT);
query.add(new TermQuery(new Term(field, "w5")), BooleanClause.Occur.MUST_NOT);
@ -213,6 +291,7 @@ public class TestBoolean2 extends LuceneTestCase {
@Test
public void testQueries08() throws Exception {
BooleanQuery.Builder query = new BooleanQuery.Builder();
query.setDisableCoord(random().nextBoolean());
query.add(new TermQuery(new Term(field, "w3")), BooleanClause.Occur.MUST);
query.add(new TermQuery(new Term(field, "xx")), BooleanClause.Occur.SHOULD);
query.add(new TermQuery(new Term(field, "w5")), BooleanClause.Occur.MUST_NOT);
@ -223,6 +302,7 @@ public class TestBoolean2 extends LuceneTestCase {
@Test
public void testQueries09() throws Exception {
BooleanQuery.Builder query = new BooleanQuery.Builder();
query.setDisableCoord(random().nextBoolean());
query.add(new TermQuery(new Term(field, "w3")), BooleanClause.Occur.MUST);
query.add(new TermQuery(new Term(field, "xx")), BooleanClause.Occur.MUST);
query.add(new TermQuery(new Term(field, "w2")), BooleanClause.Occur.MUST);
@ -234,6 +314,7 @@ public class TestBoolean2 extends LuceneTestCase {
@Test
public void testQueries10() throws Exception {
BooleanQuery.Builder query = new BooleanQuery.Builder();
query.setDisableCoord(random().nextBoolean());
query.add(new TermQuery(new Term(field, "w3")), BooleanClause.Occur.MUST);
query.add(new TermQuery(new Term(field, "xx")), BooleanClause.Occur.MUST);
query.add(new TermQuery(new Term(field, "w2")), BooleanClause.Occur.MUST);
@ -241,16 +322,19 @@ public class TestBoolean2 extends LuceneTestCase {
int[] expDocNrs = {2, 3};
Similarity oldSimilarity = searcher.getSimilarity(true);
try {
searcher.setSimilarity(new ClassicSimilarity(){
Similarity newSimilarity = new ClassicSimilarity() {
@Override
public float coord(int overlap, int maxOverlap) {
return overlap / ((float)maxOverlap - 1);
}
});
};
try {
searcher.setSimilarity(newSimilarity);
singleSegmentSearcher.setSimilarity(newSimilarity);
queriesTest(query.build(), expDocNrs);
} finally {
searcher.setSimilarity(oldSimilarity);
singleSegmentSearcher.setSimilarity(oldSimilarity);
}
}
@ -282,15 +366,11 @@ public class TestBoolean2 extends LuceneTestCase {
searcher.setSimilarity(new ClassicSimilarity()); // restore
}
TopFieldCollector collector = TopFieldCollector.create(sort, 1000,
false, true, true);
// check diff (randomized) scorers (from AssertingSearcher) produce the same results
TopFieldCollector collector = TopFieldCollector.create(sort, 1000, false, true, true);
searcher.search(q1, collector);
ScoreDoc[] hits1 = collector.topDocs().scoreDocs;
collector = TopFieldCollector.create(sort, 1000,
false, true, true);
collector = TopFieldCollector.create(sort, 1000, false, true, true);
searcher.search(q1, collector);
ScoreDoc[] hits2 = collector.topDocs().scoreDocs;
tot+=hits2.length;
@ -301,6 +381,16 @@ public class TestBoolean2 extends LuceneTestCase {
q3.add(new PrefixQuery(new Term("field2", "b")), BooleanClause.Occur.SHOULD);
TopDocs hits4 = bigSearcher.search(q3.build(), 1);
assertEquals(mulFactor*collector.totalHits + NUM_EXTRA_DOCS/2, hits4.totalHits);
// test diff (randomized) scorers produce the same results on bigSearcher as well
collector = TopFieldCollector.create(sort, 1000 * mulFactor, false, true, true);
bigSearcher.search(q1, collector);
hits1 = collector.topDocs().scoreDocs;
collector = TopFieldCollector.create(sort, 1000 * mulFactor, false, true, true);
bigSearcher.search(q1, collector);
hits2 = collector.topDocs().scoreDocs;
CheckHits.checkEqual(q1, hits1, hits2);
}
} catch (Exception e) {

View File

@ -40,7 +40,6 @@ import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.SnapshotDeletionPolicy;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.ThreadedIndexingAndSearchingTestCase;
import org.apache.lucene.index.TrackingIndexWriter;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.NRTCachingDirectory;
import org.apache.lucene.util.IOUtils;
@ -57,7 +56,7 @@ public class TestControlledRealTimeReopenThread extends ThreadedIndexingAndSearc
// Is guaranteed to reflect deletes:
private SearcherManager nrtDeletes;
private TrackingIndexWriter genWriter;
private IndexWriter genWriter;
private ControlledRealTimeReopenThread<IndexSearcher> nrtDeletesThread;
private ControlledRealTimeReopenThread<IndexSearcher> nrtNoDeletesThread;
@ -219,7 +218,7 @@ public class TestControlledRealTimeReopenThread extends ThreadedIndexingAndSearc
System.out.println("TEST: make SearcherManager maxReopenSec=" + maxReopenSec + " minReopenSec=" + minReopenSec);
}
genWriter = new TrackingIndexWriter(writer);
genWriter = writer;
final SearcherFactory sf = new SearcherFactory() {
@Override
@ -311,9 +310,8 @@ public class TestControlledRealTimeReopenThread extends ThreadedIndexingAndSearc
final CountDownLatch latch = new CountDownLatch(1);
final CountDownLatch signal = new CountDownLatch(1);
LatchedIndexWriter _writer = new LatchedIndexWriter(d, conf, latch, signal);
final TrackingIndexWriter writer = new TrackingIndexWriter(_writer);
final SearcherManager manager = new SearcherManager(_writer, false, false, null);
LatchedIndexWriter writer = new LatchedIndexWriter(d, conf, latch, signal);
final SearcherManager manager = new SearcherManager(writer, false, false, null);
Document doc = new Document();
doc.add(newTextField("test", "test", Field.Store.YES));
writer.addDocument(doc);
@ -334,7 +332,7 @@ public class TestControlledRealTimeReopenThread extends ThreadedIndexingAndSearc
}
};
t.start();
_writer.waitAfterUpdate = true; // wait in addDocument to let some reopens go through
writer.waitAfterUpdate = true; // wait in addDocument to let some reopens go through
final long lastGen = writer.updateDocument(new Term("foo", "bar"), doc); // once this returns the doc is already reflected in the last reopen
assertFalse(manager.isSearcherCurrent()); // false since there is a delete in the queue
@ -373,7 +371,7 @@ public class TestControlledRealTimeReopenThread extends ThreadedIndexingAndSearc
}
thread.close();
thread.join();
_writer.close();
writer.close();
IOUtils.close(manager, d);
}
@ -389,14 +387,13 @@ public class TestControlledRealTimeReopenThread extends ThreadedIndexingAndSearc
super(d, conf);
this.latch = latch;
this.signal = signal;
}
@Override
public void updateDocument(Term term,
public long updateDocument(Term term,
Iterable<? extends IndexableField> doc)
throws IOException {
super.updateDocument(term, doc);
long result = super.updateDocument(term, doc);
try {
if (waitAfterUpdate) {
signal.countDown();
@ -405,6 +402,7 @@ public class TestControlledRealTimeReopenThread extends ThreadedIndexingAndSearc
} catch (InterruptedException e) {
throw new ThreadInterruptedException(e);
}
return result;
}
}
@ -483,9 +481,8 @@ public class TestControlledRealTimeReopenThread extends ThreadedIndexingAndSearc
config.setOpenMode(IndexWriterConfig.OpenMode.CREATE_OR_APPEND);
final IndexWriter iw = new IndexWriter(dir, config);
SearcherManager sm = new SearcherManager(iw, new SearcherFactory());
final TrackingIndexWriter tiw = new TrackingIndexWriter(iw);
ControlledRealTimeReopenThread<IndexSearcher> controlledRealTimeReopenThread =
new ControlledRealTimeReopenThread<>(tiw, sm, maxStaleSecs, 0);
new ControlledRealTimeReopenThread<>(iw, sm, maxStaleSecs, 0);
controlledRealTimeReopenThread.setDaemon(true);
controlledRealTimeReopenThread.start();
@ -517,7 +514,7 @@ public class TestControlledRealTimeReopenThread extends ThreadedIndexingAndSearc
d.add(new TextField("count", i + "", Field.Store.NO));
d.add(new TextField("content", content, Field.Store.YES));
long start = System.currentTimeMillis();
long l = tiw.addDocument(d);
long l = iw.addDocument(d);
controlledRealTimeReopenThread.waitForGeneration(l);
long wait = System.currentTimeMillis() - start;
assertTrue("waited too long for generation " + wait,

View File

@ -296,6 +296,7 @@ public class TestSimpleExplanations extends BaseExplanationTestCase {
outerQuery.add(new TermQuery(new Term(FIELD, "w1")), BooleanClause.Occur.MUST);
BooleanQuery.Builder innerQuery = new BooleanQuery.Builder();
innerQuery.setDisableCoord(random().nextBoolean());
innerQuery.add(new TermQuery(new Term(FIELD, "qq")), BooleanClause.Occur.SHOULD);
BooleanQuery.Builder childLeft = new BooleanQuery.Builder();
@ -317,6 +318,7 @@ public class TestSimpleExplanations extends BaseExplanationTestCase {
outerQuery.add(new TermQuery(new Term(FIELD, "w1")), BooleanClause.Occur.MUST);
BooleanQuery.Builder innerQuery = new BooleanQuery.Builder();
innerQuery.setDisableCoord(random().nextBoolean());
innerQuery.add(new TermQuery(new Term(FIELD, "qq")), BooleanClause.Occur.SHOULD);
BooleanQuery.Builder childLeft = new BooleanQuery.Builder();
@ -338,6 +340,7 @@ public class TestSimpleExplanations extends BaseExplanationTestCase {
outerQuery.add(new TermQuery(new Term(FIELD, "w1")), BooleanClause.Occur.MUST);
BooleanQuery.Builder innerQuery = new BooleanQuery.Builder();
innerQuery.setDisableCoord(random().nextBoolean());
innerQuery.add(new TermQuery(new Term(FIELD, "qq")), BooleanClause.Occur.SHOULD);
BooleanQuery.Builder childLeft = new BooleanQuery.Builder();
@ -359,6 +362,7 @@ public class TestSimpleExplanations extends BaseExplanationTestCase {
outerQuery.add(new TermQuery(new Term(FIELD, "w1")), BooleanClause.Occur.MUST);
BooleanQuery.Builder innerQuery = new BooleanQuery.Builder();
innerQuery.setDisableCoord(random().nextBoolean());
innerQuery.add(new TermQuery(new Term(FIELD, "qq")), BooleanClause.Occur.SHOULD);
BooleanQuery.Builder childLeft = new BooleanQuery.Builder();
@ -377,6 +381,7 @@ public class TestSimpleExplanations extends BaseExplanationTestCase {
}
public void testBQ11() throws Exception {
BooleanQuery.Builder query = new BooleanQuery.Builder();
query.setDisableCoord(random().nextBoolean());
query.add(new TermQuery(new Term(FIELD, "w1")), BooleanClause.Occur.SHOULD);
TermQuery boostedQuery = new TermQuery(new Term(FIELD, "w1"));
query.add(new BoostQuery(boostedQuery, 1000), BooleanClause.Occur.SHOULD);
@ -385,21 +390,21 @@ public class TestSimpleExplanations extends BaseExplanationTestCase {
}
public void testBQ14() throws Exception {
BooleanQuery.Builder q = new BooleanQuery.Builder();
q.setDisableCoord(true);
q.setDisableCoord(random().nextBoolean());
q.add(new TermQuery(new Term(FIELD, "QQQQQ")), BooleanClause.Occur.SHOULD);
q.add(new TermQuery(new Term(FIELD, "w1")), BooleanClause.Occur.SHOULD);
qtest(q.build(), new int[] { 0,1,2,3 });
}
public void testBQ15() throws Exception {
BooleanQuery.Builder q = new BooleanQuery.Builder();
q.setDisableCoord(true);
q.setDisableCoord(random().nextBoolean());
q.add(new TermQuery(new Term(FIELD, "QQQQQ")), BooleanClause.Occur.MUST_NOT);
q.add(new TermQuery(new Term(FIELD, "w1")), BooleanClause.Occur.SHOULD);
qtest(q.build(), new int[] { 0,1,2,3 });
}
public void testBQ16() throws Exception {
BooleanQuery.Builder q = new BooleanQuery.Builder();
q.setDisableCoord(true);
q.setDisableCoord(random().nextBoolean());
q.add(new TermQuery(new Term(FIELD, "QQQQQ")), BooleanClause.Occur.SHOULD);
BooleanQuery.Builder booleanQuery = new BooleanQuery.Builder();
@ -411,7 +416,7 @@ public class TestSimpleExplanations extends BaseExplanationTestCase {
}
public void testBQ17() throws Exception {
BooleanQuery.Builder q = new BooleanQuery.Builder();
q.setDisableCoord(true);
q.setDisableCoord(random().nextBoolean());
q.add(new TermQuery(new Term(FIELD, "w2")), BooleanClause.Occur.SHOULD);
BooleanQuery.Builder booleanQuery = new BooleanQuery.Builder();
@ -431,6 +436,7 @@ public class TestSimpleExplanations extends BaseExplanationTestCase {
public void testBQ20() throws Exception {
BooleanQuery.Builder q = new BooleanQuery.Builder();
q.setDisableCoord(random().nextBoolean());
q.setMinimumNumberShouldMatch(2);
q.add(new TermQuery(new Term(FIELD, "QQQQQ")), BooleanClause.Occur.SHOULD);
q.add(new TermQuery(new Term(FIELD, "yy")), BooleanClause.Occur.SHOULD);
@ -441,6 +447,16 @@ public class TestSimpleExplanations extends BaseExplanationTestCase {
qtest(q.build(), new int[] { 0,3 });
}
public void testBQ21() throws Exception {
BooleanQuery.Builder q = new BooleanQuery.Builder();
q.setDisableCoord(random().nextBoolean());
q.add(new TermQuery(new Term(FIELD, "yy")), BooleanClause.Occur.SHOULD);
q.add(new TermQuery(new Term(FIELD, "zz")), BooleanClause.Occur.SHOULD);
qtest(q.build(), new int[] { 1,2,3 });
}
public void testBQ23() throws Exception {
BooleanQuery.Builder query = new BooleanQuery.Builder();
@ -488,6 +504,7 @@ public class TestSimpleExplanations extends BaseExplanationTestCase {
}
public void testMultiFieldBQ3() throws Exception {
BooleanQuery.Builder query = new BooleanQuery.Builder();
query.setDisableCoord(random().nextBoolean());
query.add(new TermQuery(new Term(FIELD, "yy")), BooleanClause.Occur.SHOULD);
query.add(new TermQuery(new Term(ALTFIELD, "w3")), BooleanClause.Occur.MUST);
@ -495,6 +512,7 @@ public class TestSimpleExplanations extends BaseExplanationTestCase {
}
public void testMultiFieldBQ4() throws Exception {
BooleanQuery.Builder outerQuery = new BooleanQuery.Builder();
outerQuery.setDisableCoord(random().nextBoolean());
outerQuery.add(new TermQuery(new Term(FIELD, "w1")), BooleanClause.Occur.SHOULD);
BooleanQuery.Builder innerQuery = new BooleanQuery.Builder();
@ -506,6 +524,7 @@ public class TestSimpleExplanations extends BaseExplanationTestCase {
}
public void testMultiFieldBQ5() throws Exception {
BooleanQuery.Builder outerQuery = new BooleanQuery.Builder();
outerQuery.setDisableCoord(random().nextBoolean());
outerQuery.add(new TermQuery(new Term(FIELD, "w1")), BooleanClause.Occur.SHOULD);
BooleanQuery.Builder innerQuery = new BooleanQuery.Builder();

View File

@ -0,0 +1,126 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.search;
import java.util.Arrays;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.util.TestUtil;
import org.junit.BeforeClass;
import org.junit.Assume;
/**
* subclass of TestSimpleExplanations that adds a lot of filler docs which will be ignored at query time.
* These filler docs will either all be empty in which case the queries will be unmodified, or they will
* all use terms from same set of source data as our regular docs (to emphasis the DocFreq factor in scoring),
* in which case the queries will be wrapped so they can be excluded.
*/
public class TestSimpleExplanationsWithFillerDocs extends TestSimpleExplanations {
/** num of empty docs injected between every doc in the index */
private static final int NUM_FILLER_DOCS = BooleanScorer.SIZE;
/** num of empty docs injected prior to the first doc in the (main) index */
private static int PRE_FILLER_DOCS;
/**
* If non-null then the filler docs are not empty, and need to be filtered out from queries
* using this as both field name &amp; field value
*/
public static String EXTRA = null;
private static final Document EMPTY_DOC = new Document();
/**
* Replaces the index created by our superclass with a new one that includes a lot of docs filler docs.
* {@link #qtest} will account for these extra filler docs.
* @see #qtest
*/
@BeforeClass
public static void replaceIndex() throws Exception {
EXTRA = random().nextBoolean() ? null : "extra";
PRE_FILLER_DOCS = TestUtil.nextInt(random(), 0, (NUM_FILLER_DOCS / 2));
// free up what our super class created that we won't be using
reader.close();
directory.close();
directory = newDirectory();
try (RandomIndexWriter writer = new RandomIndexWriter(random(), directory, newIndexWriterConfig(analyzer).setMergePolicy(newLogMergePolicy()))) {
for (int filler = 0; filler < PRE_FILLER_DOCS; filler++) {
writer.addDocument(makeFillerDoc());
}
for (int i = 0; i < docFields.length; i++) {
writer.addDocument(createDoc(i));
for (int filler = 0; filler < NUM_FILLER_DOCS; filler++) {
writer.addDocument(makeFillerDoc());
}
}
reader = writer.getReader();
searcher = newSearcher(reader);
}
}
private static Document makeFillerDoc() {
if (null == EXTRA) {
return EMPTY_DOC;
}
Document doc = createDoc(TestUtil.nextInt(random(), 0, docFields.length-1));
doc.add(newStringField(EXTRA, EXTRA, Field.Store.NO));
return doc;
}
/**
* Adjusts <code>expDocNrs</code> based on the filler docs injected in the index,
* and if neccessary wraps the <code>q</code> in a BooleanQuery that will filter out all
* filler docs using the {@link #EXTRA} field.
*
* @see #replaceIndex
*/
@Override
public void qtest(Query q, int[] expDocNrs) throws Exception {
expDocNrs = Arrays.copyOf(expDocNrs, expDocNrs.length);
for (int i=0; i < expDocNrs.length; i++) {
expDocNrs[i] = PRE_FILLER_DOCS + ((NUM_FILLER_DOCS + 1) * expDocNrs[i]);
}
if (null != EXTRA) {
BooleanQuery.Builder builder = new BooleanQuery.Builder();
builder.add(new BooleanClause(q, BooleanClause.Occur.MUST));
builder.add(new BooleanClause(new TermQuery(new Term(EXTRA, EXTRA)), BooleanClause.Occur.MUST_NOT));
q = builder.build();
}
super.qtest(q, expDocNrs);
}
public void testMA1() throws Exception {
Assume.assumeNotNull("test is not viable with empty filler docs", EXTRA);
super.testMA1();
}
public void testMA2() throws Exception {
Assume.assumeNotNull("test is not viable with empty filler docs", EXTRA);
super.testMA2();
}
}

View File

@ -581,14 +581,14 @@ public class DirectoryTaxonomyWriter implements TaxonomyWriter {
}
@Override
public synchronized void commit() throws IOException {
public synchronized long commit() throws IOException {
ensureOpen();
// LUCENE-4972: if we always call setCommitData, we create empty commits
String epochStr = indexWriter.getCommitData().get(INDEX_EPOCH);
if (epochStr == null || Long.parseLong(epochStr, 16) != indexEpoch) {
indexWriter.setCommitData(combinedCommitData(indexWriter.getCommitData()));
}
indexWriter.commit();
return indexWriter.commit();
}
/** Combine original user data with the taxonomy epoch. */
@ -616,14 +616,14 @@ public class DirectoryTaxonomyWriter implements TaxonomyWriter {
* See {@link IndexWriter#prepareCommit}.
*/
@Override
public synchronized void prepareCommit() throws IOException {
public synchronized long prepareCommit() throws IOException {
ensureOpen();
// LUCENE-4972: if we always call setCommitData, we create empty commits
String epochStr = indexWriter.getCommitData().get(INDEX_EPOCH);
if (epochStr == null || Long.parseLong(epochStr, 16) != indexEpoch) {
indexWriter.setCommitData(combinedCommitData(indexWriter.getCommitData()));
}
indexWriter.prepareCommit();
return indexWriter.prepareCommit();
}
@Override

View File

@ -130,14 +130,15 @@ public class RandomIndexWriter implements Closeable {
* Adds a Document.
* @see IndexWriter#addDocument(Iterable)
*/
public <T extends IndexableField> void addDocument(final Iterable<T> doc) throws IOException {
public <T extends IndexableField> long addDocument(final Iterable<T> doc) throws IOException {
LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig());
long seqNo;
if (r.nextInt(5) == 3) {
// TODO: maybe, we should simply buffer up added docs
// (but we need to clone them), and only when
// getReader, commit, etc. are called, we do an
// addDocuments? Would be better testing.
w.addDocuments(new Iterable<Iterable<T>>() {
seqNo = w.addDocuments(new Iterable<Iterable<T>>() {
@Override
public Iterator<Iterable<T>> iterator() {
@ -167,10 +168,12 @@ public class RandomIndexWriter implements Closeable {
}
});
} else {
w.addDocument(doc);
seqNo = w.addDocument(doc);
}
maybeFlushOrCommit();
return seqNo;
}
private void maybeFlushOrCommit() throws IOException {
@ -195,26 +198,29 @@ public class RandomIndexWriter implements Closeable {
}
}
public void addDocuments(Iterable<? extends Iterable<? extends IndexableField>> docs) throws IOException {
public long addDocuments(Iterable<? extends Iterable<? extends IndexableField>> docs) throws IOException {
LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig());
w.addDocuments(docs);
long seqNo = w.addDocuments(docs);
maybeFlushOrCommit();
return seqNo;
}
public void updateDocuments(Term delTerm, Iterable<? extends Iterable<? extends IndexableField>> docs) throws IOException {
public long updateDocuments(Term delTerm, Iterable<? extends Iterable<? extends IndexableField>> docs) throws IOException {
LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig());
w.updateDocuments(delTerm, docs);
long seqNo = w.updateDocuments(delTerm, docs);
maybeFlushOrCommit();
return seqNo;
}
/**
* Updates a document.
* @see IndexWriter#updateDocument(Term, Iterable)
*/
public <T extends IndexableField> void updateDocument(Term t, final Iterable<T> doc) throws IOException {
public <T extends IndexableField> long updateDocument(Term t, final Iterable<T> doc) throws IOException {
LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig());
long seqNo;
if (r.nextInt(5) == 3) {
w.updateDocuments(t, new Iterable<Iterable<T>>() {
seqNo = w.updateDocuments(t, new Iterable<Iterable<T>>() {
@Override
public Iterator<Iterable<T>> iterator() {
@ -243,49 +249,51 @@ public class RandomIndexWriter implements Closeable {
}
});
} else {
w.updateDocument(t, doc);
seqNo = w.updateDocument(t, doc);
}
maybeFlushOrCommit();
return seqNo;
}
public void addIndexes(Directory... dirs) throws IOException {
public long addIndexes(Directory... dirs) throws IOException {
LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig());
w.addIndexes(dirs);
return w.addIndexes(dirs);
}
public void addIndexes(CodecReader... readers) throws IOException {
public long addIndexes(CodecReader... readers) throws IOException {
LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig());
w.addIndexes(readers);
return w.addIndexes(readers);
}
public void updateNumericDocValue(Term term, String field, Long value) throws IOException {
public long updateNumericDocValue(Term term, String field, Long value) throws IOException {
LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig());
w.updateNumericDocValue(term, field, value);
return w.updateNumericDocValue(term, field, value);
}
public void updateBinaryDocValue(Term term, String field, BytesRef value) throws IOException {
public long updateBinaryDocValue(Term term, String field, BytesRef value) throws IOException {
LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig());
w.updateBinaryDocValue(term, field, value);
return w.updateBinaryDocValue(term, field, value);
}
public void updateDocValues(Term term, Field... updates) throws IOException {
public long updateDocValues(Term term, Field... updates) throws IOException {
LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig());
w.updateDocValues(term, updates);
return w.updateDocValues(term, updates);
}
public void deleteDocuments(Term term) throws IOException {
public long deleteDocuments(Term term) throws IOException {
LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig());
w.deleteDocuments(term);
return w.deleteDocuments(term);
}
public void deleteDocuments(Query q) throws IOException {
public long deleteDocuments(Query q) throws IOException {
LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig());
w.deleteDocuments(q);
return w.deleteDocuments(q);
}
public void commit() throws IOException {
public long commit() throws IOException {
LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig());
w.commit();
return w.commit();
}
public int numDocs() {
@ -296,8 +304,8 @@ public class RandomIndexWriter implements Closeable {
return w.maxDoc();
}
public void deleteAll() throws IOException {
w.deleteAll();
public long deleteAll() throws IOException {
return w.deleteAll();
}
public DirectoryReader getReader() throws IOException {

View File

@ -71,22 +71,26 @@ public abstract class BaseExplanationTestCase extends LuceneTestCase {
public static void beforeClassTestExplanations() throws Exception {
directory = newDirectory();
analyzer = new MockAnalyzer(random());
RandomIndexWriter writer= new RandomIndexWriter(random(), directory, newIndexWriterConfig(analyzer).setMergePolicy(newLogMergePolicy()));
for (int i = 0; i < docFields.length; i++) {
Document doc = new Document();
doc.add(newStringField(KEY, ""+i, Field.Store.NO));
doc.add(new SortedDocValuesField(KEY, new BytesRef(""+i)));
Field f = newTextField(FIELD, docFields[i], Field.Store.NO);
f.setBoost(i);
doc.add(f);
doc.add(newTextField(ALTFIELD, docFields[i], Field.Store.NO));
writer.addDocument(doc);
try (RandomIndexWriter writer = new RandomIndexWriter(random(), directory, newIndexWriterConfig(analyzer).setMergePolicy(newLogMergePolicy()))) {
for (int i = 0; i < docFields.length; i++) {
writer.addDocument(createDoc(i));
}
reader = writer.getReader();
searcher = newSearcher(reader);
}
reader = writer.getReader();
writer.close();
searcher = newSearcher(reader);
}
public static Document createDoc(int index) {
Document doc = new Document();
doc.add(newStringField(KEY, ""+index, Field.Store.NO));
doc.add(new SortedDocValuesField(KEY, new BytesRef(""+index)));
Field f = newTextField(FIELD, docFields[index], Field.Store.NO);
f.setBoost(index);
doc.add(f);
doc.add(newTextField(ALTFIELD, docFields[index], Field.Store.NO));
return doc;
}
protected static final String[] docFields = {
"w1 w2 w3 w4 w5",
"w1 w3 w2 w3 zz",
@ -94,8 +98,19 @@ public abstract class BaseExplanationTestCase extends LuceneTestCase {
"w1 w3 xx w2 yy w3 zz"
};
/** check the expDocNrs first, then check the query (and the explanations) */
/**
* check the expDocNrs match and have scores that match the explanations.
* Query may be randomly wrapped in a BooleanQuery with a term that matches no documents in
* order to trigger coord logic.
*/
public void qtest(Query q, int[] expDocNrs) throws Exception {
if (random().nextBoolean()) {
BooleanQuery.Builder bq = new BooleanQuery.Builder();
bq.setDisableCoord(random().nextBoolean());
bq.add(q, BooleanClause.Occur.SHOULD);
bq.add(new TermQuery(new Term("NEVER","MATCH")), BooleanClause.Occur.SHOULD);
q = bq.build();
}
CheckHits.checkHitCollector(random(), q, FIELD, searcher, expDocNrs);
}

View File

@ -0,0 +1,120 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.search;
import java.io.IOException;
import java.util.Set;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.Term;
import junit.framework.AssertionFailedError;
/**
* Tests that the {@link BaseExplanationTestCase} helper code, as well as
* {@link CheckHits#checkNoMatchExplanations} are checking what they are suppose to.
*/
public class TestBaseExplanationTestCase extends BaseExplanationTestCase {
public void testQueryNoMatchWhenExpected() throws Exception {
expectThrows(AssertionFailedError.class, () -> {
qtest(new TermQuery(new Term(FIELD, "BOGUS")), new int[] { 3 /* none */ });
});
}
public void testQueryMatchWhenNotExpected() throws Exception {
expectThrows(AssertionFailedError.class, () -> {
qtest(new TermQuery(new Term(FIELD, "w1")), new int[] { 0, 1 /*, 2, 3 */ });
});
}
public void testIncorrectExplainScores() throws Exception {
// sanity check what a real TermQuery matches
qtest(new TermQuery(new Term(FIELD, "zz")), new int[] { 1, 3 });
// ensure when the Explanations are broken, we get an error about those matches
expectThrows(AssertionFailedError.class, () -> {
qtest(new BrokenExplainTermQuery(new Term(FIELD, "zz"), false, true), new int[] { 1, 3 });
});
}
public void testIncorrectExplainMatches() throws Exception {
// sanity check what a real TermQuery matches
qtest(new TermQuery(new Term(FIELD, "zz")), new int[] { 1, 3 });
// ensure when the Explanations are broken, we get an error about the non matches
expectThrows(AssertionFailedError.class, () -> {
CheckHits.checkNoMatchExplanations(new BrokenExplainTermQuery(new Term(FIELD, "zz"), true, false),
FIELD, searcher, new int[] { 1, 3 });
});
}
public static final class BrokenExplainTermQuery extends TermQuery {
public final boolean toggleExplainMatch;
public final boolean breakExplainScores;
public BrokenExplainTermQuery(Term t, boolean toggleExplainMatch, boolean breakExplainScores) {
super(t);
this.toggleExplainMatch = toggleExplainMatch;
this.breakExplainScores = breakExplainScores;
}
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
return new BrokenExplainWeight(this, super.createWeight(searcher,needsScores));
}
}
public static final class BrokenExplainWeight extends Weight {
final Weight in;
public BrokenExplainWeight(BrokenExplainTermQuery q, Weight in) {
super(q);
this.in = in;
}
public BulkScorer bulkScorer(LeafReaderContext context) throws IOException {
return in.bulkScorer(context);
}
public Explanation explain(LeafReaderContext context, int doc) throws IOException {
BrokenExplainTermQuery q = (BrokenExplainTermQuery) this.getQuery();
Explanation result = in.explain(context, doc);
if (result.isMatch()) {
if (q.breakExplainScores) {
result = Explanation.match(-1F * result.getValue(), "Broken Explanation Score", result);
}
if (q.toggleExplainMatch) {
result = Explanation.noMatch("Broken Explanation Matching", result);
}
} else {
if (q.toggleExplainMatch) {
result = Explanation.match(-42.0F, "Broken Explanation Matching", result);
}
}
return result;
}
public void extractTerms(Set<Term> terms) {
in.extractTerms(terms);
}
public float getValueForNormalization() throws IOException {
return in.getValueForNormalization();
}
public void normalize(float norm, float boost) {
in.normalize(norm, boost);
}
public Scorer scorer(LeafReaderContext context) throws IOException {
return in.scorer(context);
}
}
}

View File

@ -150,6 +150,8 @@ New Features
* SOLR-7123: '/update/json/docs' path supports nested documents (noble)
* SOLR-8610: Resolve variables in encryptKeyFile of DIH's JdbcDataSource (Kristine Jetzke via Mikhail Khludnev)
Bug Fixes
----------------------
@ -249,6 +251,10 @@ Bug Fixes
* SOLR-8940: Fix group.sort option (hossman)
* SOLR-8612: closing JDBC Statement on failures in DataImportHandler (DIH) (Kristine Jetzke via Mikhail Khludnev)
* SOLR-8676: keep LOG4J_CONFIG in solr.cmd (Kristine Jetzke via Mikhail Khludnev)
Optimizations
----------------------
* SOLR-8722: Don't force a full ZkStateReader refresh on every Overseer operation.
@ -277,9 +283,6 @@ Optimizations
* SOLR-8744: Overseer operations performed with fine grained mutual exclusion (noble, Scott Blum)
* SOLR-9140: Replace zk polling in ZkController with CollectionStateWatchers
(Alan Woodward)
Other Changes
----------------------
* SOLR-7516: Improve javadocs for JavaBinCodec, ObjectResolver and enforce the single-usage policy.
@ -357,6 +360,8 @@ Other Changes
* SOLR-9081: Make SolrTestCaseJ4.beforeClass() / .afterClass() public so it
works with Mockito (Georg Sorst, Alan Woodward)
* SOLR-8445: fix line separator in log4j.properties files (Ahmet Arslan via Mikhail Khludnev)
================== 6.0.1 ==================
(No Changes)

View File

@ -657,7 +657,6 @@ IF "%STOP_KEY%"=="" set STOP_KEY=solrrocks
@REM so that we can write logs for examples to %SOLR_HOME%\..\logs
set "SOLR_LOGS_DIR=%SOLR_SERVER_DIR%\logs"
set "EXAMPLE_DIR=%SOLR_TIP%\example"
set LOG4J_CONFIG=
set TMP=!SOLR_HOME:%EXAMPLE_DIR%=!
IF NOT "%TMP%"=="%SOLR_HOME%" (
set "SOLR_LOGS_DIR=%SOLR_HOME%\..\logs"

View File

@ -4,7 +4,7 @@ log4j.rootLogger=INFO, CONSOLE
log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender
log4j.appender.CONSOLE.Target=System.err
log4j.appender.CONSOLE.layout=org.apache.solr.util.SolrLogLayout
log4j.appender.CONSOLE.layout.ConversionPattern=%-5p - %d{yyyy-MM-dd HH:mm:ss.SSS}; %C; %m\n
log4j.appender.CONSOLE.layout.ConversionPattern=%-5p - %d{yyyy-MM-dd HH:mm:ss.SSS}; %C; %m%n
log4j.logger.org.apache.zookeeper=WARN
log4j.logger.org.apache.hadoop=WARN

View File

@ -29,14 +29,12 @@ import javax.naming.InitialContext;
import javax.naming.NamingException;
import java.io.FileInputStream;
import java.io.FileReader;
import java.io.IOException;
import java.io.InputStreamReader;
import java.io.Reader;
import java.lang.invoke.MethodHandles;
import java.math.BigDecimal;
import java.math.BigInteger;
import java.nio.charset.StandardCharsets;
import java.sql.*;
import java.util.*;
import java.util.concurrent.Callable;
@ -60,6 +58,8 @@ public class JdbcDataSource extends
private long connLastUsed = 0;
private Connection conn;
private ResultSetIterator resultSetIterator;
private Map<String, Integer> fieldNameVsType = new HashMap<>();
@ -71,7 +71,7 @@ public class JdbcDataSource extends
@Override
public void init(Context context, Properties initProps) {
initProps = decryptPwd(initProps);
initProps = decryptPwd(context, initProps);
Object o = initProps.get(CONVERT_TYPE);
if (o != null)
convertType = Boolean.parseBoolean(o.toString());
@ -112,8 +112,8 @@ public class JdbcDataSource extends
}
}
private Properties decryptPwd(Properties initProps) {
String encryptionKey = initProps.getProperty("encryptKeyFile");
private Properties decryptPwd(Context context, Properties initProps) {
String encryptionKey = context.replaceTokens(initProps.getProperty("encryptKeyFile"));
if (initProps.getProperty("password") != null && encryptionKey != null) {
// this means the password is encrypted and use the file to decode it
try {
@ -276,15 +276,19 @@ public class JdbcDataSource extends
@Override
public Iterator<Map<String, Object>> getData(String query) {
ResultSetIterator r = new ResultSetIterator(query);
return r.getIterator();
if (resultSetIterator != null) {
resultSetIterator.close();
resultSetIterator = null;
}
resultSetIterator = new ResultSetIterator(query);
return resultSetIterator.getIterator();
}
private void logError(String msg, Exception e) {
LOG.warn(msg, e);
}
private List<String> readFieldNames(ResultSetMetaData metaData)
protected List<String> readFieldNames(ResultSetMetaData metaData)
throws SQLException {
List<String> colNames = new ArrayList<>();
int count = metaData.getColumnCount();
@ -299,35 +303,38 @@ public class JdbcDataSource extends
private Statement stmt = null;
private List<String> colNames;
private Iterator<Map<String, Object>> rSetIterator;
public ResultSetIterator(String query) {
final List<String> colNames;
try {
Connection c = getConnection();
stmt = createStatement(c);
stmt = createStatement(c, batchSize, maxRows);
LOG.debug("Executing SQL: " + query);
long start = System.nanoTime();
resultSet = executeStatement(stmt, query);
LOG.trace("Time taken for sql :"
+ TimeUnit.MILLISECONDS.convert(System.nanoTime() - start, TimeUnit.NANOSECONDS));
colNames = readFieldNames(resultSet.getMetaData());
setColNames(resultSet);
} catch (Exception e) {
close();
wrapAndThrow(SEVERE, e, "Unable to execute query: " + query);
return;
}
if (resultSet == null) {
close();
rSetIterator = new ArrayList<Map<String, Object>>().iterator();
return;
}
rSetIterator = createIterator(stmt, resultSet, convertType, colNames, fieldNameVsType);
rSetIterator = createIterator(convertType, fieldNameVsType);
}
protected Statement createStatement(Connection c) throws SQLException {
protected Statement createStatement(final Connection c, final int batchSize, final int maxRows)
throws SQLException {
Statement statement = c.createStatement(ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY);
statement.setFetchSize(batchSize);
statement.setMaxRows(maxRows);
@ -340,19 +347,26 @@ public class JdbcDataSource extends
}
return null;
}
protected void setColNames(final ResultSet resultSet) throws SQLException {
if (resultSet != null) {
colNames = readFieldNames(resultSet.getMetaData());
} else {
colNames = Collections.emptyList();
}
}
protected Iterator<Map<String,Object>> createIterator(Statement stmt, ResultSet resultSet, boolean convertType,
List<String> colNames, Map<String,Integer> fieldNameVsType) {
protected Iterator<Map<String,Object>> createIterator(final boolean convertType,
final Map<String,Integer> fieldNameVsType) {
return new Iterator<Map<String,Object>>() {
@Override
public boolean hasNext() {
return hasnext(resultSet, stmt);
return hasnext();
}
@Override
public Map<String,Object> next() {
return getARow(resultSet, convertType, colNames, fieldNameVsType);
return getARow(convertType, fieldNameVsType);
}
@Override
@ -363,17 +377,16 @@ public class JdbcDataSource extends
protected Map<String,Object> getARow(ResultSet resultSet, boolean convertType, List<String> colNames,
Map<String,Integer> fieldNameVsType) {
if (resultSet == null)
protected Map<String,Object> getARow(boolean convertType, Map<String,Integer> fieldNameVsType) {
if (getResultSet() == null)
return null;
Map<String, Object> result = new HashMap<>();
for (String colName : colNames) {
for (String colName : getColNames()) {
try {
if (!convertType) {
// Use underlying database's type information except for BigDecimal and BigInteger
// which cannot be serialized by JavaBin/XML. See SOLR-6165
Object value = resultSet.getObject(colName);
Object value = getResultSet().getObject(colName);
if (value instanceof BigDecimal || value instanceof BigInteger) {
result.put(colName, value.toString());
} else {
@ -387,28 +400,28 @@ public class JdbcDataSource extends
type = Types.VARCHAR;
switch (type) {
case Types.INTEGER:
result.put(colName, resultSet.getInt(colName));
result.put(colName, getResultSet().getInt(colName));
break;
case Types.FLOAT:
result.put(colName, resultSet.getFloat(colName));
result.put(colName, getResultSet().getFloat(colName));
break;
case Types.BIGINT:
result.put(colName, resultSet.getLong(colName));
result.put(colName, getResultSet().getLong(colName));
break;
case Types.DOUBLE:
result.put(colName, resultSet.getDouble(colName));
result.put(colName, getResultSet().getDouble(colName));
break;
case Types.DATE:
result.put(colName, resultSet.getTimestamp(colName));
result.put(colName, getResultSet().getTimestamp(colName));
break;
case Types.BOOLEAN:
result.put(colName, resultSet.getBoolean(colName));
result.put(colName, getResultSet().getBoolean(colName));
break;
case Types.BLOB:
result.put(colName, resultSet.getBytes(colName));
result.put(colName, getResultSet().getBytes(colName));
break;
default:
result.put(colName, resultSet.getString(colName));
result.put(colName, getResultSet().getString(colName));
break;
}
} catch (SQLException e) {
@ -419,11 +432,13 @@ public class JdbcDataSource extends
return result;
}
protected boolean hasnext(ResultSet resultSet, Statement stmt) {
if (resultSet == null)
protected boolean hasnext() {
if (getResultSet() == null) {
close();
return false;
}
try {
if (resultSet.next()) {
if (getResultSet().next()) {
return true;
} else {
close();
@ -438,15 +453,15 @@ public class JdbcDataSource extends
protected void close() {
try {
if (resultSet != null)
resultSet.close();
if (stmt != null)
stmt.close();
if (getResultSet() != null)
getResultSet().close();
if (getStatement() != null)
getStatement().close();
} catch (Exception e) {
logError("Exception while closing result set", e);
} finally {
resultSet = null;
stmt = null;
setResultSet(null);
setStatement(null);
}
}
@ -454,6 +469,31 @@ public class JdbcDataSource extends
return rSetIterator;
}
protected final Statement getStatement() {
return stmt;
}
protected final void setStatement(Statement stmt) {
this.stmt = stmt;
}
protected final ResultSet getResultSet() {
return resultSet;
}
protected final void setResultSet(ResultSet resultSet) {
this.resultSet = resultSet;
}
protected final List<String> getColNames() {
return colNames;
}
protected final void setColNames(List<String> colNames) {
this.colNames = colNames;
}
}
protected Connection getConnection() throws Exception {
@ -488,6 +528,9 @@ public class JdbcDataSource extends
@Override
public void close() {
if (resultSetIterator != null) {
resultSetIterator.close();
}
try {
closeConnection();
} finally {

View File

@ -4,6 +4,6 @@ log4j.rootLogger=INFO, CONSOLE
log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender
log4j.appender.CONSOLE.Target=System.err
log4j.appender.CONSOLE.layout=org.apache.solr.util.SolrLogLayout
log4j.appender.CONSOLE.layout.ConversionPattern=%-5p - %d{yyyy-MM-dd HH:mm:ss.SSS}; %C; %m\n
log4j.appender.CONSOLE.layout.ConversionPattern=%-5p - %d{yyyy-MM-dd HH:mm:ss.SSS}; %C; %m%n
log4j.logger.org.apache.zookeeper=WARN

View File

@ -253,7 +253,7 @@ public class TestDocBuilder2 extends AbstractDataImportHandlerTestCase {
@Test
@Ignore("Fix Me. See SOLR-4103.")
public void testFileListEntityProcessor_lastIndexTime() throws Exception {
File tmpdir = File.createTempFile("test", "tmp", createTempDir().toFile());
File tmpdir = createTempDir().toFile();
Map<String, String> params = createMap("baseDir", tmpdir.getAbsolutePath());

View File

@ -15,18 +15,23 @@
* limitations under the License.
*/
package org.apache.solr.handler.dataimport;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
import org.apache.solr.common.util.SuppressForbidden;
import org.junit.Ignore;
import org.junit.Test;
import java.io.File;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.nio.file.Files;
import java.text.SimpleDateFormat;
import java.util.*;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Date;
import java.util.HashSet;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Set;
import org.apache.solr.common.util.SuppressForbidden;
import org.junit.Test;
/**
* <p>
@ -65,9 +70,7 @@ public class TestFileListEntityProcessor extends AbstractDataImportHandlerTestCa
@Test
public void testBiggerSmallerFiles() throws IOException {
File tmpdir = File.createTempFile("test", "tmp", createTempDir().toFile());
Files.delete(tmpdir.toPath());
tmpdir.mkdir();
File tmpdir = createTempDir().toFile();
long minLength = Long.MAX_VALUE;
String smallestFile = "";

View File

@ -17,16 +17,21 @@
package org.apache.solr.handler.dataimport;
import java.io.File;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.nio.file.Files;
import java.sql.Connection;
import java.sql.Driver;
import java.sql.DriverManager;
import java.sql.ResultSet;
import java.sql.ResultSetMetaData;
import java.sql.SQLException;
import java.sql.Statement;
import java.util.*;
import javax.sql.DataSource;
import org.apache.solr.handler.dataimport.JdbcDataSource.ResultSetIterator;
import org.easymock.EasyMock;
import org.easymock.IMocksControl;
import org.junit.After;
@ -132,15 +137,10 @@ public class TestJdbcDataSource extends AbstractDataImportHandlerTestCase {
@Test
public void testRetrieveFromJndiWithCredentialsWithEncryptedPwd() throws Exception {
MockInitialContextFactory.bind("java:comp/env/jdbc/JndiDB", dataSource);
File tmpdir = File.createTempFile("test", "tmp", createTempDir().toFile());
Files.delete(tmpdir.toPath());
tmpdir.mkdir();
byte[] content = "secret".getBytes(StandardCharsets.UTF_8);
createFile(tmpdir, "enckeyfile.txt", content, false);
props.put(JdbcDataSource.JNDI_NAME, "java:comp/env/jdbc/JndiDB");
props.put("user", "Fred");
props.put("encryptKeyFile", new File(tmpdir, "enckeyfile.txt").getAbsolutePath());
props.put("encryptKeyFile", createEncryptionKeyFile());
props.put("password", "U2FsdGVkX18QMjY0yfCqlfBMvAB4d3XkwY96L7gfO2o=");
props.put("holdability", "HOLD_CURSORS_OVER_COMMIT");
EasyMock.expect(dataSource.getConnection("Fred", "MyPassword")).andReturn(
@ -158,6 +158,32 @@ public class TestJdbcDataSource extends AbstractDataImportHandlerTestCase {
assertSame("connection", conn, connection);
}
@Test
public void testRetrieveFromJndiWithCredentialsWithEncryptedAndResolvedPwd() throws Exception {
MockInitialContextFactory.bind("java:comp/env/jdbc/JndiDB", dataSource);
props.put(JdbcDataSource.JNDI_NAME, "java:comp/env/jdbc/JndiDB");
props.put("user", "Fred");
props.put("encryptKeyFile", "${foo.bar}");
props.put("password", "U2FsdGVkX18QMjY0yfCqlfBMvAB4d3XkwY96L7gfO2o=");
EasyMock.expect(dataSource.getConnection("Fred", "MyPassword")).andReturn(
connection);
Map<String,Object> values = new HashMap<>();
values.put("bar", createEncryptionKeyFile());
context.getVariableResolver().addNamespace("foo", values);
jdbcDataSource.init(context, props);
connection.setAutoCommit(false);
mockControl.replay();
jdbcDataSource.getConnection();
mockControl.verify();
}
@Test
public void testRetrieveFromJndiFailureNotHidden() throws Exception {
@ -201,6 +227,177 @@ public class TestJdbcDataSource extends AbstractDataImportHandlerTestCase {
mockControl.verify();
}
@Test
public void testClosesStatementWhenExceptionThrownOnExecuteQuery() throws Exception {
MockInitialContextFactory.bind("java:comp/env/jdbc/JndiDB", dataSource);
props.put(JdbcDataSource.JNDI_NAME, "java:comp/env/jdbc/JndiDB");
EasyMock.expect(dataSource.getConnection()).andReturn(connection);
jdbcDataSource.init(context, props);
connection.setAutoCommit(false);
SQLException sqlException = new SQLException("fake");
Statement statement = mockControl.createMock(Statement.class);
EasyMock.expect(connection.createStatement(ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY))
.andReturn(statement);
statement.setFetchSize(500);
statement.setMaxRows(0);
EasyMock.expect(statement.execute("query")).andThrow(sqlException);
statement.close();
mockControl.replay();
try {
jdbcDataSource.getData("query");
fail("exception expected");
} catch (DataImportHandlerException ex) {
assertSame(sqlException, ex.getCause());
}
mockControl.verify();
}
@Test
public void testClosesStatementWhenResultSetNull() throws Exception {
MockInitialContextFactory.bind("java:comp/env/jdbc/JndiDB", dataSource);
props.put(JdbcDataSource.JNDI_NAME, "java:comp/env/jdbc/JndiDB");
EasyMock.expect(dataSource.getConnection()).andReturn(connection);
jdbcDataSource.init(context, props);
connection.setAutoCommit(false);
Statement statement = mockControl.createMock(Statement.class);
EasyMock.expect(connection.createStatement(ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY))
.andReturn(statement);
statement.setFetchSize(500);
statement.setMaxRows(0);
EasyMock.expect(statement.execute("query")).andReturn(false);
statement.close();
mockControl.replay();
jdbcDataSource.getData("query");
mockControl.verify();
}
@Test
public void testClosesStatementWhenHasNextCalledAndResultSetNull() throws Exception {
MockInitialContextFactory.bind("java:comp/env/jdbc/JndiDB", dataSource);
props.put(JdbcDataSource.JNDI_NAME, "java:comp/env/jdbc/JndiDB");
EasyMock.expect(dataSource.getConnection()).andReturn(connection);
jdbcDataSource.init(context, props);
connection.setAutoCommit(false);
Statement statement = mockControl.createMock(Statement.class);
EasyMock.expect(connection.createStatement(ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY))
.andReturn(statement);
statement.setFetchSize(500);
statement.setMaxRows(0);
EasyMock.expect(statement.execute("query")).andReturn(true);
ResultSet resultSet = mockControl.createMock(ResultSet.class);
EasyMock.expect(statement.getResultSet()).andReturn(resultSet);
ResultSetMetaData metaData = mockControl.createMock(ResultSetMetaData.class);
EasyMock.expect(resultSet.getMetaData()).andReturn(metaData);
EasyMock.expect(metaData.getColumnCount()).andReturn(0);
statement.close();
mockControl.replay();
Iterator<Map<String,Object>> data = jdbcDataSource.getData("query");
ResultSetIterator resultSetIterator = (ResultSetIterator) data.getClass().getDeclaredField("this$1").get(data);
resultSetIterator.setResultSet(null);
data.hasNext();
mockControl.verify();
}
@Test
public void testClosesResultSetAndStatementWhenDataSourceIsClosed() throws Exception {
MockInitialContextFactory.bind("java:comp/env/jdbc/JndiDB", dataSource);
props.put(JdbcDataSource.JNDI_NAME, "java:comp/env/jdbc/JndiDB");
EasyMock.expect(dataSource.getConnection()).andReturn(connection);
jdbcDataSource.init(context, props);
connection.setAutoCommit(false);
Statement statement = mockControl.createMock(Statement.class);
EasyMock.expect(connection.createStatement(ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY))
.andReturn(statement);
statement.setFetchSize(500);
statement.setMaxRows(0);
EasyMock.expect(statement.execute("query")).andReturn(true);
ResultSet resultSet = mockControl.createMock(ResultSet.class);
EasyMock.expect(statement.getResultSet()).andReturn(resultSet);
ResultSetMetaData metaData = mockControl.createMock(ResultSetMetaData.class);
EasyMock.expect(resultSet.getMetaData()).andReturn(metaData);
EasyMock.expect(metaData.getColumnCount()).andReturn(0);
resultSet.close();
statement.close();
connection.commit();
connection.close();
mockControl.replay();
jdbcDataSource.getData("query");
jdbcDataSource.close();
mockControl.verify();
}
@Test
public void testClosesCurrentResultSetIteratorWhenNewOneIsCreated() throws Exception {
MockInitialContextFactory.bind("java:comp/env/jdbc/JndiDB", dataSource);
props.put(JdbcDataSource.JNDI_NAME, "java:comp/env/jdbc/JndiDB");
EasyMock.expect(dataSource.getConnection()).andReturn(connection);
jdbcDataSource.init(context, props);
connection.setAutoCommit(false);
Statement statement = mockControl.createMock(Statement.class);
EasyMock.expect(connection.createStatement(ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY))
.andReturn(statement);
statement.setFetchSize(500);
statement.setMaxRows(0);
EasyMock.expect(statement.execute("query")).andReturn(true);
ResultSet resultSet = mockControl.createMock(ResultSet.class);
EasyMock.expect(statement.getResultSet()).andReturn(resultSet);
ResultSetMetaData metaData = mockControl.createMock(ResultSetMetaData.class);
EasyMock.expect(resultSet.getMetaData()).andReturn(metaData);
EasyMock.expect(metaData.getColumnCount()).andReturn(0);
resultSet.close();
statement.close();
EasyMock.expect(connection.createStatement(ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY))
.andReturn(statement);
statement.setFetchSize(500);
statement.setMaxRows(0);
EasyMock.expect(statement.execute("other query")).andReturn(false);
statement.close();
mockControl.replay();
jdbcDataSource.getData("query");
jdbcDataSource.getData("other query");
mockControl.verify();
}
@Test
public void testRetrieveFromDriverManager() throws Exception {
DriverManager.registerDriver(driver);
@ -228,7 +425,6 @@ public class TestJdbcDataSource extends AbstractDataImportHandlerTestCase {
DriverManager.deregisterDriver(driver);
}
}
@Test
@Ignore("Needs a Mock database server to work")
public void testBasic() throws Exception {
@ -267,4 +463,11 @@ public class TestJdbcDataSource extends AbstractDataImportHandlerTestCase {
assertEquals(Float.class, msrp.getClass());
assertEquals(Long.class, trim_id.getClass());
}
private String createEncryptionKeyFile() throws IOException {
File tmpdir = createTempDir().toFile();
byte[] content = "secret".getBytes(StandardCharsets.UTF_8);
createFile(tmpdir, "enckeyfile.txt", content, false);
return new File(tmpdir, "enckeyfile.txt").getAbsolutePath();
}
}

View File

@ -27,7 +27,6 @@ import java.nio.charset.StandardCharsets;
import java.nio.file.Path;
import java.util.*;
import java.util.concurrent.Callable;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
@ -683,23 +682,35 @@ public final class ZkController {
InterruptedException {
publishNodeAsDown(getNodeName());
// now wait till the updates are in our state
long now = System.nanoTime();
long timeout = now + TimeUnit.NANOSECONDS.convert(WAIT_DOWN_STATES_TIMEOUT_SECONDS, TimeUnit.SECONDS);
boolean foundStates = true;
Set<String> collections = cc.getLocalCollections();
CountDownLatch latch = new CountDownLatch(collections.size());
for (String collection : collections) {
zkStateReader.registerCollectionStateWatcher(collection, (nodes, state) -> {
for (Replica replica : state.getReplicasOnNode(getNodeName())) {
if (replica.getState() != Replica.State.DOWN)
return false;
while (System.nanoTime() < timeout) {
ClusterState clusterState = zkStateReader.getClusterState();
Map<String, DocCollection> collections = clusterState.getCollectionsMap();
for (Map.Entry<String, DocCollection> entry : collections.entrySet()) {
DocCollection collection = entry.getValue();
Collection<Slice> slices = collection.getSlices();
for (Slice slice : slices) {
Collection<Replica> replicas = slice.getReplicas();
for (Replica replica : replicas) {
if (getNodeName().equals(replica.getNodeName()) && replica.getState() != Replica.State.DOWN) {
foundStates = false;
}
}
}
latch.countDown();
return true;
});
}
}
if (latch.await(WAIT_DOWN_STATES_TIMEOUT_SECONDS, TimeUnit.SECONDS) == false) {
// TODO should we abort here?
if (foundStates) {
Thread.sleep(1000);
break;
}
Thread.sleep(1000);
}
if (!foundStates) {
log.warn("Timed out waiting to see all nodes published as DOWN in our cluster state.");
}
@ -1355,7 +1366,7 @@ public final class ZkController {
return zkStateReader;
}
private void doGetShardIdAndNodeNameProcess(CoreDescriptor cd) throws InterruptedException {
private void doGetShardIdAndNodeNameProcess(CoreDescriptor cd) {
final String coreNodeName = cd.getCloudDescriptor().getCoreNodeName();
if (coreNodeName != null) {
@ -1367,45 +1378,58 @@ public final class ZkController {
}
}
private void waitForCoreNodeName(CoreDescriptor descriptor) throws InterruptedException {
log.info("Waiting for coreNodeName for core {} in collection {} to be assigned",
descriptor.getName(), descriptor.getCollectionName());
final String thisNode = getNodeName();
try {
zkStateReader.waitForState(descriptor.getCollectionName(), 320, TimeUnit.SECONDS, (n, c) -> {
if (c == null)
return false;
for (Replica replica : c.getReplicasOnNode(thisNode)) {
if (descriptor.getName().equals(replica.getCoreName())) {
descriptor.getCloudDescriptor().setCoreNodeName(replica.getName());
return true;
private void waitForCoreNodeName(CoreDescriptor descriptor) {
int retryCount = 320;
log.info("look for our core node name");
while (retryCount-- > 0) {
Map<String, Slice> slicesMap = zkStateReader.getClusterState()
.getSlicesMap(descriptor.getCloudDescriptor().getCollectionName());
if (slicesMap != null) {
for (Slice slice : slicesMap.values()) {
for (Replica replica : slice.getReplicas()) {
// TODO: for really large clusters, we could 'index' on this
String nodeName = replica.getStr(ZkStateReader.NODE_NAME_PROP);
String core = replica.getStr(ZkStateReader.CORE_NAME_PROP);
String msgNodeName = getNodeName();
String msgCore = descriptor.getName();
if (msgNodeName.equals(nodeName) && core.equals(msgCore)) {
descriptor.getCloudDescriptor()
.setCoreNodeName(replica.getName());
return;
}
}
}
return false;
});
} catch (TimeoutException e) {
throw new SolrException(ErrorCode.SERVER_ERROR, "Timed out getting coreNodeName for " + descriptor.getName());
}
try {
Thread.sleep(1000);
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
}
}
}
private void waitForShardId(CoreDescriptor cd) throws InterruptedException {
private void waitForShardId(CoreDescriptor cd) {
log.info("waiting to find shard id in clusterstate for " + cd.getName());
final String thisNode = getNodeName();
try {
zkStateReader.waitForState(cd.getCollectionName(), 320, TimeUnit.SECONDS, (n, c) -> {
if (c == null)
return false;
String shardId = c.getShardId(thisNode, cd.getName());
if (shardId != null) {
cd.getCloudDescriptor().setShardId(shardId);
return true;
}
return false;
});
}
catch (TimeoutException e) {
throw new SolrException(ErrorCode.SERVER_ERROR, "Timed out getting shard id for core: " + cd.getName());
int retryCount = 320;
while (retryCount-- > 0) {
final String shardId = zkStateReader.getClusterState().getShardId(cd.getCollectionName(), getNodeName(), cd.getName());
if (shardId != null) {
cd.getCloudDescriptor().setShardId(shardId);
return;
}
try {
Thread.sleep(1000);
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
}
}
throw new SolrException(ErrorCode.SERVER_ERROR,
"Could not get shard id for core: " + cd.getName());
}
@ -1419,7 +1443,7 @@ public final class ZkController {
return coreNodeName;
}
public void preRegister(CoreDescriptor cd) throws InterruptedException {
public void preRegister(CoreDescriptor cd) {
String coreNodeName = getCoreNodeName(cd);

View File

@ -16,6 +16,17 @@
*/
package org.apache.solr.core;
import static com.google.common.base.Preconditions.checkNotNull;
import static java.util.Collections.EMPTY_MAP;
import static org.apache.solr.common.params.CommonParams.AUTHC_PATH;
import static org.apache.solr.common.params.CommonParams.AUTHZ_PATH;
import static org.apache.solr.common.params.CommonParams.COLLECTIONS_HANDLER_PATH;
import static org.apache.solr.common.params.CommonParams.CONFIGSETS_HANDLER_PATH;
import static org.apache.solr.common.params.CommonParams.CORES_HANDLER_PATH;
import static org.apache.solr.common.params.CommonParams.INFO_HANDLER_PATH;
import static org.apache.solr.common.params.CommonParams.ZK_PATH;
import static org.apache.solr.security.AuthenticationPlugin.AUTHENTICATION_PLUGIN_PROP;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.nio.file.Path;
@ -26,15 +37,12 @@ import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Properties;
import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
import java.util.stream.Collectors;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Maps;
import org.apache.http.auth.AuthSchemeProvider;
import org.apache.http.client.CredentialsProvider;
import org.apache.http.config.Lookup;
@ -48,7 +56,6 @@ import org.apache.solr.cloud.Overseer;
import org.apache.solr.cloud.ZkController;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.IOUtils;
@ -76,16 +83,8 @@ import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static com.google.common.base.Preconditions.checkNotNull;
import static java.util.Collections.EMPTY_MAP;
import static org.apache.solr.common.params.CommonParams.AUTHC_PATH;
import static org.apache.solr.common.params.CommonParams.AUTHZ_PATH;
import static org.apache.solr.common.params.CommonParams.COLLECTIONS_HANDLER_PATH;
import static org.apache.solr.common.params.CommonParams.CONFIGSETS_HANDLER_PATH;
import static org.apache.solr.common.params.CommonParams.CORES_HANDLER_PATH;
import static org.apache.solr.common.params.CommonParams.INFO_HANDLER_PATH;
import static org.apache.solr.common.params.CommonParams.ZK_PATH;
import static org.apache.solr.security.AuthenticationPlugin.AUTHENTICATION_PLUGIN_PROP;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Maps;
/**
@ -845,7 +844,6 @@ public class CoreContainer {
return core;
} catch (Exception e) {
SolrZkClient.checkInterrupted(e);
coreInitFailures.put(dcore.getName(), new CoreLoadFailure(dcore, e));
log.error("Error creating core [{}]: {}", dcore.getName(), e.getMessage(), e);
final SolrException solrException = new SolrException(ErrorCode.SERVER_ERROR, "Unable to create core [" + dcore.getName() + "]", e);
@ -895,17 +893,6 @@ public class CoreContainer {
}
/**
* @return a Set containing the names of all collections with a core hosted in this container
*/
public Set<String> getLocalCollections() {
Set<String> collections = getCoreDescriptors().stream()
.filter(cd -> cd.getCollectionName() != null)
.map(CoreDescriptor::getCollectionName)
.collect(Collectors.toSet());
return collections;
}
/**
* Returns an immutable Map of Exceptions that occured when initializing
* SolrCores (either at startup, or do to runtime requests to create cores)

View File

@ -301,10 +301,9 @@ public class ZkCLITest extends SolrTestCaseJ4 {
@Test
public void testGetFileNotExists() throws Exception {
File tmpDir = createTempDir().toFile();
String getNode = "/getFileNotExistsNode";
File file = File.createTempFile("newfile", null, tmpDir);
File file = createTempFile("newfile", null).toFile();
String[] args = new String[] {"-zkhost", zkServer.getZkAddress(), "-cmd",
"getfile", getNode, file.getAbsolutePath()};
try {

View File

@ -15,7 +15,7 @@ log4j.appender.file.MaxBackupIndex=9
#- File to log to and log format
log4j.appender.file.File=${solr.log}/solr.log
log4j.appender.file.layout=org.apache.log4j.PatternLayout
log4j.appender.file.layout.ConversionPattern=%-5p - %d{yyyy-MM-dd HH:mm:ss.SSS}; [%X{collection} %X{shard} %X{replica} %X{core}] %C; %m\n
log4j.appender.file.layout.ConversionPattern=%-5p - %d{yyyy-MM-dd HH:mm:ss.SSS}; [%X{collection} %X{shard} %X{replica} %X{core}] %C; %m%n
log4j.logger.org.apache.zookeeper=WARN
log4j.logger.org.apache.hadoop=WARN

View File

@ -15,7 +15,7 @@ log4j.appender.file.MaxBackupIndex=9
#- File to log to and log format
log4j.appender.file.File=${solr.log}/solr.log
log4j.appender.file.layout=org.apache.log4j.EnhancedPatternLayout
log4j.appender.file.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss.SSS} %-5p (%t) [%X{collection} %X{shard} %X{replica} %X{core}] %c{1.} %m\n
log4j.appender.file.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss.SSS} %-5p (%t) [%X{collection} %X{shard} %X{replica} %X{core}] %c{1.} %m%n
log4j.logger.org.apache.zookeeper=WARN
log4j.logger.org.apache.hadoop=WARN

View File

@ -5,7 +5,7 @@ log4j.rootLogger=INFO, stderr
log4j.appender.stderr = org.apache.log4j.ConsoleAppender
log4j.appender.stderr.Target = System.err
log4j.appender.stderr.layout = org.apache.log4j.PatternLayout
log4j.appender.stderr.layout.ConversionPattern=%-5p - %d{yyyy-MM-dd HH:mm:ss.SSS}; %C; %m\n
log4j.appender.stderr.layout.ConversionPattern=%-5p - %d{yyyy-MM-dd HH:mm:ss.SSS}; %C; %m%n
# quiet down the ZK logging for cli tools
log4j.logger.org.apache.zookeeper=WARN

View File

@ -1104,7 +1104,7 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
deleteAsyncId(requestId).process(client);
return state;
}
TimeUnit.MILLISECONDS.sleep(100);
TimeUnit.SECONDS.sleep(1);
}
return state;
}

View File

@ -25,7 +25,6 @@ import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.stream.Collectors;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
@ -260,26 +259,4 @@ public class DocCollection extends ZkNodeProps implements Iterable<Slice> {
}
return replicas;
}
/**
* Get all the replicas on a particular node
*/
public List<Replica> getReplicasOnNode(String nodeName) {
return getReplicas().stream()
.filter(replica -> replica.getNodeName().equals(nodeName))
.collect(Collectors.toList());
}
/**
* Get the shardId of a core on a specific node
*/
public String getShardId(String nodeName, String coreName) {
for (Slice slice : this) {
for (Replica replica : slice) {
if (Objects.equals(replica.getNodeName(), nodeName) && Objects.equals(replica.getCoreName(), coreName))
return slice.getName();
}
}
return null;
}
}

View File

@ -51,9 +51,8 @@ public class TestCollectionStateWatchers extends SolrCloudTestCase {
@BeforeClass
public static void startCluster() throws Exception {
configureCluster(CLUSTER_SIZE)
.addConfig("config", getFile("solrj/solr/configsets/streaming/conf").toPath())
.addConfig("config", getFile("solrj/solr/collection1/conf").toPath())
.configure();
cluster.getSolrClient().connect();
}
@AfterClass
@ -260,7 +259,7 @@ public class TestCollectionStateWatchers extends SolrCloudTestCase {
final CloudSolrClient client = cluster.getSolrClient();
Future<Boolean> future = waitInBackground("stateformat1", 30, TimeUnit.SECONDS,
Future<Boolean> future = waitInBackground("stateformat1", 10, TimeUnit.SECONDS,
(n, c) -> DocCollection.isFullyActive(n, c, 1, 1));
CollectionAdminRequest.createCollection("stateformat1", "config", 1, 1).setStateFormat(1)

View File

@ -139,7 +139,7 @@ import static com.google.common.base.Preconditions.checkNotNull;
})
@SuppressSysoutChecks(bugUrl = "Solr dumps tons of logs to console.")
@SuppressFileSystems("ExtrasFS") // might be ok, the failures with e.g. nightly runs might be "normal"
@RandomizeSSL()
@RandomizeSSL(ssl=0.0,reason="https://issues.apache.org/jira/browse/SOLR-9189 - temp disable randomized ssl to get to bottom of recent explosion of timeout related failures in jenkins builds")
public abstract class SolrTestCaseJ4 extends LuceneTestCase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());