mirror of
https://github.com/apache/lucene.git
synced 2025-03-01 13:59:12 +00:00
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/lucene-solr
This commit is contained in:
commit
d74db527bf
@ -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
|
||||
|
@ -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
|
||||
|
@ -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() {
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
@ -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) {
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
@ -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 (> 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 <= 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 <= 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();
|
||||
}
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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;
|
||||
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -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) {
|
||||
|
@ -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 + ")";
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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() + ")";
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
|
@ -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");
|
||||
|
@ -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();
|
||||
|
||||
|
@ -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();
|
||||
}
|
||||
}
|
@ -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");
|
||||
|
@ -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();
|
||||
|
||||
|
@ -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
|
||||
|
@ -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) {
|
||||
|
@ -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,
|
||||
|
@ -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();
|
||||
|
@ -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 & 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();
|
||||
}
|
||||
|
||||
|
||||
}
|
@ -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
|
||||
|
@ -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 {
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
@ -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)
|
||||
|
||||
|
@ -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"
|
||||
|
@ -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
|
||||
|
@ -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 {
|
||||
|
@ -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
|
||||
|
@ -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());
|
||||
|
||||
|
@ -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 = "";
|
||||
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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)
|
||||
|
@ -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 {
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
}
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
@ -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)
|
||||
|
@ -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());
|
||||
|
Loading…
x
Reference in New Issue
Block a user