mirror of https://github.com/apache/lucene.git
Delete TimeLimitingCollector after deprecation (#13243)
Co-authored-by: Kaival Parikh <kaivalp2000@gmail.com>
This commit is contained in:
parent
ce978d7646
commit
d6eb126871
|
@ -192,6 +192,10 @@ enum.
|
||||||
|
|
||||||
`IOContext.READ` has been removed, it should be replaced with `IOContext.DEFAULT`.
|
`IOContext.READ` has been removed, it should be replaced with `IOContext.DEFAULT`.
|
||||||
|
|
||||||
|
### TimeLimitingCollector removed (GITHUB#13243)
|
||||||
|
|
||||||
|
`TimeLimitingCollector` has been removed, use `IndexSearcher#setTimeout(QueryTimeout)` to time out queries instead.
|
||||||
|
|
||||||
## Migration from Lucene 9.0 to Lucene 9.1
|
## Migration from Lucene 9.0 to Lucene 9.1
|
||||||
|
|
||||||
### Test framework package migration and module (LUCENE-10301)
|
### Test framework package migration and module (LUCENE-10301)
|
||||||
|
|
|
@ -1,325 +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.search;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import org.apache.lucene.index.LeafReaderContext;
|
|
||||||
import org.apache.lucene.index.QueryTimeout;
|
|
||||||
import org.apache.lucene.util.Counter;
|
|
||||||
import org.apache.lucene.util.SuppressForbidden;
|
|
||||||
import org.apache.lucene.util.ThreadInterruptedException;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* The {@link TimeLimitingCollector} is used to timeout search requests that take longer than the
|
|
||||||
* maximum allowed search time limit. After this time is exceeded, the search thread is stopped by
|
|
||||||
* throwing a {@link TimeExceededException}.
|
|
||||||
*
|
|
||||||
* @deprecated Use {@link IndexSearcher#setTimeout(QueryTimeout)} to time out queries.
|
|
||||||
* @see org.apache.lucene.index.ExitableDirectoryReader
|
|
||||||
*/
|
|
||||||
@Deprecated
|
|
||||||
public class TimeLimitingCollector implements Collector {
|
|
||||||
|
|
||||||
/** Thrown when elapsed search time exceeds allowed search time. */
|
|
||||||
@SuppressWarnings("serial")
|
|
||||||
public static class TimeExceededException extends RuntimeException {
|
|
||||||
private final long timeAllowed;
|
|
||||||
private final long timeElapsed;
|
|
||||||
private final int lastDocCollected;
|
|
||||||
|
|
||||||
private TimeExceededException(long timeAllowed, long timeElapsed, int lastDocCollected) {
|
|
||||||
super(
|
|
||||||
"Elapsed time: "
|
|
||||||
+ timeElapsed
|
|
||||||
+ ". Exceeded allowed search time: "
|
|
||||||
+ timeAllowed
|
|
||||||
+ " ms.");
|
|
||||||
this.timeAllowed = timeAllowed;
|
|
||||||
this.timeElapsed = timeElapsed;
|
|
||||||
this.lastDocCollected = lastDocCollected;
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Returns allowed time (milliseconds). */
|
|
||||||
public long getTimeAllowed() {
|
|
||||||
return timeAllowed;
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Returns elapsed time (milliseconds). */
|
|
||||||
public long getTimeElapsed() {
|
|
||||||
return timeElapsed;
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Returns last doc (absolute doc id) that was collected when the search time exceeded. */
|
|
||||||
public int getLastDocCollected() {
|
|
||||||
return lastDocCollected;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private long t0 = Long.MIN_VALUE;
|
|
||||||
private long timeout = Long.MIN_VALUE;
|
|
||||||
private Collector collector;
|
|
||||||
private final Counter clock;
|
|
||||||
private final long ticksAllowed;
|
|
||||||
private boolean greedy = false;
|
|
||||||
private int docBase;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Create a TimeLimitedCollector wrapper over another {@link Collector} with a specified timeout.
|
|
||||||
*
|
|
||||||
* @param collector the wrapped {@link Collector}
|
|
||||||
* @param clock the timer clock
|
|
||||||
* @param ticksAllowed max time allowed for collecting hits after which {@link
|
|
||||||
* TimeExceededException} is thrown
|
|
||||||
*/
|
|
||||||
public TimeLimitingCollector(final Collector collector, Counter clock, final long ticksAllowed) {
|
|
||||||
this.collector = collector;
|
|
||||||
this.clock = clock;
|
|
||||||
this.ticksAllowed = ticksAllowed;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Sets the baseline for this collector. By default the collectors baseline is initialized once
|
|
||||||
* the first reader is passed to the collector. To include operations executed in prior to the
|
|
||||||
* actual document collection set the baseline through this method in your prelude.
|
|
||||||
*
|
|
||||||
* <p>Example usage:
|
|
||||||
*
|
|
||||||
* <pre class="prettyprint">
|
|
||||||
* Counter clock = ...;
|
|
||||||
* long baseline = clock.get();
|
|
||||||
* // ... prepare search
|
|
||||||
* TimeLimitingCollector collector = new TimeLimitingCollector(c, clock, numTicks);
|
|
||||||
* collector.setBaseline(baseline);
|
|
||||||
* indexSearcher.search(query, collector);
|
|
||||||
* </pre>
|
|
||||||
*
|
|
||||||
* @see #setBaseline()
|
|
||||||
*/
|
|
||||||
public void setBaseline(long clockTime) {
|
|
||||||
t0 = clockTime;
|
|
||||||
timeout = t0 + ticksAllowed;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Syntactic sugar for {@link #setBaseline(long)} using {@link Counter#get()} on the clock passed
|
|
||||||
* to the constructor.
|
|
||||||
*/
|
|
||||||
public void setBaseline() {
|
|
||||||
setBaseline(clock.get());
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Checks if this time limited collector is greedy in collecting the last hit. A non greedy
|
|
||||||
* collector, upon a timeout, would throw a {@link TimeExceededException} without allowing the
|
|
||||||
* wrapped collector to collect current doc. A greedy one would first allow the wrapped hit
|
|
||||||
* collector to collect current doc and only then throw a {@link TimeExceededException}. However,
|
|
||||||
* if the timeout is detected in {@link #getLeafCollector} then no current document is collected.
|
|
||||||
*
|
|
||||||
* @see #setGreedy(boolean)
|
|
||||||
*/
|
|
||||||
public boolean isGreedy() {
|
|
||||||
return greedy;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Sets whether this time limited collector is greedy.
|
|
||||||
*
|
|
||||||
* @param greedy true to make this time limited greedy
|
|
||||||
* @see #isGreedy()
|
|
||||||
*/
|
|
||||||
public void setGreedy(boolean greedy) {
|
|
||||||
this.greedy = greedy;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
|
|
||||||
this.docBase = context.docBase;
|
|
||||||
if (Long.MIN_VALUE == t0) {
|
|
||||||
setBaseline();
|
|
||||||
}
|
|
||||||
final long time = clock.get();
|
|
||||||
if (time - timeout > 0L) {
|
|
||||||
throw new TimeExceededException(timeout - t0, time - t0, -1);
|
|
||||||
}
|
|
||||||
return new FilterLeafCollector(collector.getLeafCollector(context)) {
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void collect(int doc) throws IOException {
|
|
||||||
final long time = clock.get();
|
|
||||||
if (time - timeout > 0L) {
|
|
||||||
if (greedy) {
|
|
||||||
// System.out.println(this+" greedy: before failing, collecting doc: "+(docBase +
|
|
||||||
// doc)+" "+(time-t0));
|
|
||||||
in.collect(doc);
|
|
||||||
}
|
|
||||||
// System.out.println(this+" failing on: "+(docBase + doc)+" "+(time-t0));
|
|
||||||
throw new TimeExceededException(timeout - t0, time - t0, docBase + doc);
|
|
||||||
}
|
|
||||||
// System.out.println(this+" collecting: "+(docBase + doc)+" "+(time-t0));
|
|
||||||
in.collect(doc);
|
|
||||||
}
|
|
||||||
};
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public ScoreMode scoreMode() {
|
|
||||||
return collector.scoreMode();
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* This is so the same timer can be used with a multi-phase search process such as grouping. We
|
|
||||||
* don't want to create a new TimeLimitingCollector for each phase because that would reset the
|
|
||||||
* timer for each phase. Once time is up subsequent phases need to timeout quickly.
|
|
||||||
*
|
|
||||||
* @param collector The actual collector performing search functionality
|
|
||||||
*/
|
|
||||||
public void setCollector(Collector collector) {
|
|
||||||
this.collector = collector;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Returns the global TimerThreads {@link Counter}
|
|
||||||
*
|
|
||||||
* <p>Invoking this creates may create a new instance of {@link TimerThread} iff the global {@link
|
|
||||||
* TimerThread} has never been accessed before. The thread returned from this method is started on
|
|
||||||
* creation and will be alive unless you stop the {@link TimerThread} via {@link
|
|
||||||
* TimerThread#stopTimer()}.
|
|
||||||
*
|
|
||||||
* @return the global TimerThreads {@link Counter}
|
|
||||||
* @lucene.experimental
|
|
||||||
*/
|
|
||||||
public static Counter getGlobalCounter() {
|
|
||||||
return TimerThreadHolder.THREAD.counter;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Returns the global {@link TimerThread}.
|
|
||||||
*
|
|
||||||
* <p>Invoking this creates may create a new instance of {@link TimerThread} iff the global {@link
|
|
||||||
* TimerThread} has never been accessed before. The thread returned from this method is started on
|
|
||||||
* creation and will be alive unless you stop the {@link TimerThread} via {@link
|
|
||||||
* TimerThread#stopTimer()}.
|
|
||||||
*
|
|
||||||
* @return the global {@link TimerThread}
|
|
||||||
* @lucene.experimental
|
|
||||||
*/
|
|
||||||
public static TimerThread getGlobalTimerThread() {
|
|
||||||
return TimerThreadHolder.THREAD;
|
|
||||||
}
|
|
||||||
|
|
||||||
private static final class TimerThreadHolder {
|
|
||||||
static final TimerThread THREAD;
|
|
||||||
|
|
||||||
static {
|
|
||||||
THREAD = new TimerThread(Counter.newCounter(true));
|
|
||||||
THREAD.start();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Thread used to timeout search requests. Can be stopped completely with {@link
|
|
||||||
* TimerThread#stopTimer()}
|
|
||||||
*
|
|
||||||
* @lucene.experimental
|
|
||||||
*/
|
|
||||||
public static final class TimerThread extends Thread {
|
|
||||||
|
|
||||||
public static final String THREAD_NAME = "TimeLimitedCollector timer thread";
|
|
||||||
public static final int DEFAULT_RESOLUTION = 20;
|
|
||||||
// NOTE: we can avoid explicit synchronization here for several reasons:
|
|
||||||
// * updates to volatile long variables are atomic
|
|
||||||
// * only single thread modifies this value
|
|
||||||
// * use of volatile keyword ensures that it does not reside in
|
|
||||||
// a register, but in main memory (so that changes are visible to
|
|
||||||
// other threads).
|
|
||||||
// * visibility of changes does not need to be instantaneous, we can
|
|
||||||
// afford losing a tick or two.
|
|
||||||
//
|
|
||||||
// See section 17 of the Java Language Specification for details.
|
|
||||||
private volatile long time = 0;
|
|
||||||
private volatile boolean stop = false;
|
|
||||||
private volatile long resolution;
|
|
||||||
final Counter counter;
|
|
||||||
|
|
||||||
public TimerThread(long resolution, Counter counter) {
|
|
||||||
super(THREAD_NAME);
|
|
||||||
this.resolution = resolution;
|
|
||||||
this.counter = counter;
|
|
||||||
this.setDaemon(true);
|
|
||||||
}
|
|
||||||
|
|
||||||
public TimerThread(Counter counter) {
|
|
||||||
this(DEFAULT_RESOLUTION, counter);
|
|
||||||
}
|
|
||||||
|
|
||||||
@SuppressForbidden(reason = "Thread sleep")
|
|
||||||
@Override
|
|
||||||
public void run() {
|
|
||||||
while (!stop) {
|
|
||||||
// TODO: Use System.nanoTime() when Lucene moves to Java SE 5.
|
|
||||||
counter.addAndGet(resolution);
|
|
||||||
try {
|
|
||||||
Thread.sleep(resolution);
|
|
||||||
} catch (InterruptedException ie) {
|
|
||||||
throw new ThreadInterruptedException(ie);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Get the timer value in milliseconds. */
|
|
||||||
public long getMilliseconds() {
|
|
||||||
return time;
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Stops the timer thread */
|
|
||||||
public void stopTimer() {
|
|
||||||
stop = true;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Return the timer resolution.
|
|
||||||
*
|
|
||||||
* @see #setResolution(long)
|
|
||||||
*/
|
|
||||||
public long getResolution() {
|
|
||||||
return resolution;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Set the timer resolution. The default timer resolution is 20 milliseconds. This means that a
|
|
||||||
* search required to take no longer than 800 milliseconds may be stopped after 780 to 820
|
|
||||||
* milliseconds. <br>
|
|
||||||
* Note that:
|
|
||||||
*
|
|
||||||
* <ul>
|
|
||||||
* <li>Finer (smaller) resolution is more accurate but less efficient.
|
|
||||||
* <li>Setting resolution to less than 5 milliseconds will be silently modified to 5
|
|
||||||
* milliseconds.
|
|
||||||
* <li>Setting resolution smaller than current resolution might take effect only after current
|
|
||||||
* resolution. (Assume current resolution of 20 milliseconds is modified to 5
|
|
||||||
* milliseconds, then it can take up to 20 milliseconds for the change to have effect.
|
|
||||||
* </ul>
|
|
||||||
*/
|
|
||||||
public void setResolution(long resolution) {
|
|
||||||
this.resolution =
|
|
||||||
Math.max(
|
|
||||||
resolution,
|
|
||||||
5); // 5 milliseconds is about the minimum reasonable time for a Object.wait(long)
|
|
||||||
// call.
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -1,407 +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.search;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.util.BitSet;
|
|
||||||
import org.apache.lucene.document.Document;
|
|
||||||
import org.apache.lucene.document.Field;
|
|
||||||
import org.apache.lucene.index.IndexReader;
|
|
||||||
import org.apache.lucene.index.LeafReaderContext;
|
|
||||||
import org.apache.lucene.index.Term;
|
|
||||||
import org.apache.lucene.search.TimeLimitingCollector.TimeExceededException;
|
|
||||||
import org.apache.lucene.search.TimeLimitingCollector.TimerThread;
|
|
||||||
import org.apache.lucene.store.Directory;
|
|
||||||
import org.apache.lucene.tests.analysis.MockAnalyzer;
|
|
||||||
import org.apache.lucene.tests.index.RandomIndexWriter;
|
|
||||||
import org.apache.lucene.tests.util.LuceneTestCase;
|
|
||||||
import org.apache.lucene.tests.util.LuceneTestCase.SuppressSysoutChecks;
|
|
||||||
import org.apache.lucene.tests.util.TestUtil;
|
|
||||||
import org.apache.lucene.util.Counter;
|
|
||||||
import org.apache.lucene.util.SuppressForbidden;
|
|
||||||
import org.apache.lucene.util.ThreadInterruptedException;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Tests the {@link TimeLimitingCollector}. This test checks (1) search correctness (regardless of
|
|
||||||
* timeout), (2) expected timeout behavior, and (3) a sanity test with multiple searching threads.
|
|
||||||
*/
|
|
||||||
@SuppressSysoutChecks(bugUrl = "http://test.is.timing.sensitive.so.it.prints.instead.of.failing")
|
|
||||||
public class TestTimeLimitingCollector extends LuceneTestCase {
|
|
||||||
private static final int SLOW_DOWN = 3;
|
|
||||||
private static final long TIME_ALLOWED = 17 * SLOW_DOWN; // so searches can find about 17 docs.
|
|
||||||
|
|
||||||
// max time allowed is relaxed for multithreading tests.
|
|
||||||
// the multithread case fails when setting this to 1 (no slack) and launching many threads
|
|
||||||
// (>2000).
|
|
||||||
// but this is not a real failure, just noise.
|
|
||||||
private static final double MULTI_THREAD_SLACK = 7;
|
|
||||||
|
|
||||||
private static final int N_DOCS = 3000;
|
|
||||||
private static final int N_THREADS = 50;
|
|
||||||
|
|
||||||
private IndexSearcher searcher;
|
|
||||||
private Directory directory;
|
|
||||||
private IndexReader reader;
|
|
||||||
|
|
||||||
private final String FIELD_NAME = "body";
|
|
||||||
private Query query;
|
|
||||||
private Counter counter;
|
|
||||||
private TimerThread counterThread;
|
|
||||||
|
|
||||||
/** initializes searcher with a document set */
|
|
||||||
@Override
|
|
||||||
public void setUp() throws Exception {
|
|
||||||
super.setUp();
|
|
||||||
counter = Counter.newCounter(true);
|
|
||||||
counterThread = new TimerThread(counter);
|
|
||||||
counterThread.start();
|
|
||||||
final String[] docText = {
|
|
||||||
"docThatNeverMatchesSoWeCanRequireLastDocCollectedToBeGreaterThanZero",
|
|
||||||
"one blah three",
|
|
||||||
"one foo three multiOne",
|
|
||||||
"one foobar three multiThree",
|
|
||||||
"blueberry pancakes",
|
|
||||||
"blueberry pie",
|
|
||||||
"blueberry strudel",
|
|
||||||
"blueberry pizza",
|
|
||||||
};
|
|
||||||
directory = newDirectory();
|
|
||||||
RandomIndexWriter iw =
|
|
||||||
new RandomIndexWriter(
|
|
||||||
random(),
|
|
||||||
directory,
|
|
||||||
newIndexWriterConfig(new MockAnalyzer(random())).setMergePolicy(newLogMergePolicy()));
|
|
||||||
|
|
||||||
for (int i = 0; i < N_DOCS; i++) {
|
|
||||||
add(docText[i % docText.length], iw);
|
|
||||||
}
|
|
||||||
reader = iw.getReader();
|
|
||||||
iw.close();
|
|
||||||
searcher = newSearcher(reader);
|
|
||||||
|
|
||||||
BooleanQuery.Builder booleanQuery = new BooleanQuery.Builder();
|
|
||||||
booleanQuery.add(new TermQuery(new Term(FIELD_NAME, "one")), BooleanClause.Occur.SHOULD);
|
|
||||||
// start from 1, so that the 0th doc never matches
|
|
||||||
for (int i = 1; i < docText.length; i++) {
|
|
||||||
String[] docTextParts = docText[i].split("\\s+");
|
|
||||||
for (String docTextPart : docTextParts) { // large query so that search will be longer
|
|
||||||
booleanQuery.add(
|
|
||||||
new TermQuery(new Term(FIELD_NAME, docTextPart)), BooleanClause.Occur.SHOULD);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
query = booleanQuery.build();
|
|
||||||
|
|
||||||
// warm the searcher
|
|
||||||
searcher.search(query, 1000);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void tearDown() throws Exception {
|
|
||||||
reader.close();
|
|
||||||
directory.close();
|
|
||||||
counterThread.stopTimer();
|
|
||||||
counterThread.join();
|
|
||||||
super.tearDown();
|
|
||||||
}
|
|
||||||
|
|
||||||
private void add(String value, RandomIndexWriter iw) throws IOException {
|
|
||||||
Document d = new Document();
|
|
||||||
d.add(newTextField(FIELD_NAME, value, Field.Store.NO));
|
|
||||||
iw.addDocument(d);
|
|
||||||
}
|
|
||||||
|
|
||||||
private void search(Collector collector) throws Exception {
|
|
||||||
searcher.search(query, collector);
|
|
||||||
}
|
|
||||||
|
|
||||||
/** test search correctness with no timeout */
|
|
||||||
public void testSearch() {
|
|
||||||
doTestSearch();
|
|
||||||
}
|
|
||||||
|
|
||||||
private void doTestSearch() {
|
|
||||||
int totalResults = 0;
|
|
||||||
int totalTLCResults = 0;
|
|
||||||
try {
|
|
||||||
MyHitCollector myHc = new MyHitCollector();
|
|
||||||
search(myHc);
|
|
||||||
totalResults = myHc.hitCount();
|
|
||||||
|
|
||||||
myHc = new MyHitCollector();
|
|
||||||
long oneHour = 3600000;
|
|
||||||
long duration = TestUtil.nextLong(random(), oneHour, Long.MAX_VALUE);
|
|
||||||
Collector tlCollector = createTimedCollector(myHc, duration, false);
|
|
||||||
search(tlCollector);
|
|
||||||
totalTLCResults = myHc.hitCount();
|
|
||||||
} catch (Exception e) {
|
|
||||||
e.printStackTrace();
|
|
||||||
assertTrue("Unexpected exception: " + e, false); // ==fail
|
|
||||||
}
|
|
||||||
assertEquals("Wrong number of results!", totalResults, totalTLCResults);
|
|
||||||
}
|
|
||||||
|
|
||||||
private Collector createTimedCollector(MyHitCollector hc, long timeAllowed, boolean greedy) {
|
|
||||||
TimeLimitingCollector res = new TimeLimitingCollector(hc, counter, timeAllowed);
|
|
||||||
res.setGreedy(greedy); // set to true to make sure at least one doc is collected.
|
|
||||||
return res;
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Test that timeout is obtained, and soon enough! */
|
|
||||||
public void testTimeoutGreedy() {
|
|
||||||
doTestTimeout(false, true);
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Test that timeout is obtained, and soon enough! */
|
|
||||||
public void testTimeoutNotGreedy() {
|
|
||||||
doTestTimeout(false, false);
|
|
||||||
}
|
|
||||||
|
|
||||||
private void doTestTimeout(boolean multiThreaded, boolean greedy) {
|
|
||||||
// setup
|
|
||||||
MyHitCollector myHc = new MyHitCollector();
|
|
||||||
myHc.setSlowDown(SLOW_DOWN);
|
|
||||||
Collector tlCollector = createTimedCollector(myHc, TIME_ALLOWED, greedy);
|
|
||||||
|
|
||||||
// search: must get exception
|
|
||||||
TimeExceededException timeoutException =
|
|
||||||
expectThrows(
|
|
||||||
TimeExceededException.class,
|
|
||||||
() -> {
|
|
||||||
search(tlCollector);
|
|
||||||
});
|
|
||||||
|
|
||||||
// greediness affect last doc collected
|
|
||||||
int exceptionDoc = timeoutException.getLastDocCollected();
|
|
||||||
int lastCollected = myHc.getLastDocCollected();
|
|
||||||
|
|
||||||
// exceptionDoc == -1 means we hit the timeout in getLeafCollector:
|
|
||||||
if (exceptionDoc != -1) {
|
|
||||||
assertTrue(
|
|
||||||
"doc collected at timeout must be > 0! or == -1 but was: " + exceptionDoc,
|
|
||||||
exceptionDoc > 0);
|
|
||||||
if (greedy) {
|
|
||||||
assertTrue(
|
|
||||||
"greedy="
|
|
||||||
+ greedy
|
|
||||||
+ " exceptionDoc="
|
|
||||||
+ exceptionDoc
|
|
||||||
+ " != lastCollected="
|
|
||||||
+ lastCollected,
|
|
||||||
exceptionDoc == lastCollected);
|
|
||||||
assertTrue("greedy, but no hits found!", myHc.hitCount() > 0);
|
|
||||||
} else {
|
|
||||||
assertTrue(
|
|
||||||
"greedy="
|
|
||||||
+ greedy
|
|
||||||
+ " exceptionDoc="
|
|
||||||
+ exceptionDoc
|
|
||||||
+ " not > lastCollected="
|
|
||||||
+ lastCollected,
|
|
||||||
exceptionDoc > lastCollected);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// verify that elapsed time at exception is within valid limits
|
|
||||||
assertEquals(timeoutException.getTimeAllowed(), TIME_ALLOWED);
|
|
||||||
// a) Not too early
|
|
||||||
assertTrue(
|
|
||||||
"elapsed="
|
|
||||||
+ timeoutException.getTimeElapsed()
|
|
||||||
+ " <= (allowed-resolution)="
|
|
||||||
+ (TIME_ALLOWED - counterThread.getResolution()),
|
|
||||||
timeoutException.getTimeElapsed() > TIME_ALLOWED - counterThread.getResolution());
|
|
||||||
// b) Not too late.
|
|
||||||
// This part is problematic in a busy test system, so we just print a warning.
|
|
||||||
// We already verified that a timeout occurred, we just can't be picky about how long it
|
|
||||||
// took.
|
|
||||||
if (timeoutException.getTimeElapsed() > maxTime(multiThreaded)) {
|
|
||||||
System.out.println(
|
|
||||||
"Informative: timeout exceeded (no action required: most probably just "
|
|
||||||
+ " because the test machine is slower than usual): "
|
|
||||||
+ "lastDoc="
|
|
||||||
+ exceptionDoc
|
|
||||||
+ " ,&& allowed="
|
|
||||||
+ timeoutException.getTimeAllowed()
|
|
||||||
+ " ,&& elapsed="
|
|
||||||
+ timeoutException.getTimeElapsed()
|
|
||||||
+ " >= "
|
|
||||||
+ maxTimeStr(multiThreaded));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private long maxTime(boolean multiThreaded) {
|
|
||||||
long res =
|
|
||||||
2 * counterThread.getResolution()
|
|
||||||
+ TIME_ALLOWED
|
|
||||||
+ SLOW_DOWN; // some slack for less noise in this test
|
|
||||||
if (multiThreaded) {
|
|
||||||
res *= MULTI_THREAD_SLACK; // larger slack
|
|
||||||
}
|
|
||||||
return res;
|
|
||||||
}
|
|
||||||
|
|
||||||
private String maxTimeStr(boolean multiThreaded) {
|
|
||||||
String s =
|
|
||||||
"( "
|
|
||||||
+ "2*resolution + TIME_ALLOWED + SLOW_DOWN = "
|
|
||||||
+ "2*"
|
|
||||||
+ counterThread.getResolution()
|
|
||||||
+ " + "
|
|
||||||
+ TIME_ALLOWED
|
|
||||||
+ " + "
|
|
||||||
+ SLOW_DOWN
|
|
||||||
+ ")";
|
|
||||||
if (multiThreaded) {
|
|
||||||
s = MULTI_THREAD_SLACK + " * " + s;
|
|
||||||
}
|
|
||||||
return maxTime(multiThreaded) + " = " + s;
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Test timeout behavior when resolution is modified. */
|
|
||||||
public void testModifyResolution() {
|
|
||||||
try {
|
|
||||||
// increase and test
|
|
||||||
long resolution = 20 * TimerThread.DEFAULT_RESOLUTION; // 400
|
|
||||||
counterThread.setResolution(resolution);
|
|
||||||
assertEquals(resolution, counterThread.getResolution());
|
|
||||||
doTestTimeout(false, true);
|
|
||||||
// decrease much and test
|
|
||||||
resolution = 5;
|
|
||||||
counterThread.setResolution(resolution);
|
|
||||||
assertEquals(resolution, counterThread.getResolution());
|
|
||||||
doTestTimeout(false, true);
|
|
||||||
// return to default and test
|
|
||||||
resolution = TimerThread.DEFAULT_RESOLUTION;
|
|
||||||
counterThread.setResolution(resolution);
|
|
||||||
assertEquals(resolution, counterThread.getResolution());
|
|
||||||
doTestTimeout(false, true);
|
|
||||||
} finally {
|
|
||||||
counterThread.setResolution(TimerThread.DEFAULT_RESOLUTION);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public void testNoHits() throws IOException {
|
|
||||||
MyHitCollector myHc = new MyHitCollector();
|
|
||||||
Collector collector = createTimedCollector(myHc, -1, random().nextBoolean());
|
|
||||||
// search: must get exception
|
|
||||||
expectThrows(
|
|
||||||
TimeExceededException.class,
|
|
||||||
() -> {
|
|
||||||
BooleanQuery.Builder booleanQuery =
|
|
||||||
new BooleanQuery
|
|
||||||
.Builder(); // won't match - we only test if we check timeout when collectors are
|
|
||||||
// pulled
|
|
||||||
booleanQuery.add(new TermQuery(new Term(FIELD_NAME, "one")), BooleanClause.Occur.MUST);
|
|
||||||
booleanQuery.add(
|
|
||||||
new TermQuery(new Term(FIELD_NAME, "blueberry")), BooleanClause.Occur.MUST);
|
|
||||||
searcher.search(booleanQuery.build(), collector);
|
|
||||||
});
|
|
||||||
assertEquals(-1, myHc.getLastDocCollected());
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Test correctness with multiple searching threads. */
|
|
||||||
public void testSearchMultiThreaded() throws Exception {
|
|
||||||
doTestMultiThreads(false);
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Test correctness with multiple searching threads. */
|
|
||||||
public void testTimeoutMultiThreaded() throws Exception {
|
|
||||||
doTestMultiThreads(true);
|
|
||||||
}
|
|
||||||
|
|
||||||
private void doTestMultiThreads(final boolean withTimeout) throws Exception {
|
|
||||||
Thread[] threadArray = new Thread[N_THREADS];
|
|
||||||
final BitSet success = new BitSet(N_THREADS);
|
|
||||||
for (int i = 0; i < threadArray.length; ++i) {
|
|
||||||
final int num = i;
|
|
||||||
threadArray[num] =
|
|
||||||
new Thread() {
|
|
||||||
@Override
|
|
||||||
public void run() {
|
|
||||||
if (withTimeout) {
|
|
||||||
doTestTimeout(true, true);
|
|
||||||
} else {
|
|
||||||
doTestSearch();
|
|
||||||
}
|
|
||||||
synchronized (success) {
|
|
||||||
success.set(num);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
};
|
|
||||||
}
|
|
||||||
for (int i = 0; i < threadArray.length; ++i) {
|
|
||||||
threadArray[i].start();
|
|
||||||
}
|
|
||||||
for (int i = 0; i < threadArray.length; ++i) {
|
|
||||||
threadArray[i].join();
|
|
||||||
}
|
|
||||||
assertEquals("some threads failed!", N_THREADS, success.cardinality());
|
|
||||||
}
|
|
||||||
|
|
||||||
// counting collector that can slow down at collect().
|
|
||||||
private static class MyHitCollector extends SimpleCollector {
|
|
||||||
private final BitSet bits = new BitSet();
|
|
||||||
private int slowdown = 0;
|
|
||||||
private int lastDocCollected = -1;
|
|
||||||
private int docBase = 0;
|
|
||||||
|
|
||||||
/** amount of time to wait on each collect to simulate a long iteration */
|
|
||||||
public void setSlowDown(int milliseconds) {
|
|
||||||
slowdown = milliseconds;
|
|
||||||
}
|
|
||||||
|
|
||||||
public int hitCount() {
|
|
||||||
return bits.cardinality();
|
|
||||||
}
|
|
||||||
|
|
||||||
public int getLastDocCollected() {
|
|
||||||
return lastDocCollected;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void setScorer(Scorable scorer) throws IOException {
|
|
||||||
// scorer is not needed
|
|
||||||
}
|
|
||||||
|
|
||||||
@SuppressForbidden(reason = "Thread sleep")
|
|
||||||
@Override
|
|
||||||
public void collect(final int doc) throws IOException {
|
|
||||||
int docId = doc + docBase;
|
|
||||||
if (slowdown > 0) {
|
|
||||||
try {
|
|
||||||
Thread.sleep(slowdown);
|
|
||||||
} catch (InterruptedException ie) {
|
|
||||||
throw new ThreadInterruptedException(ie);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
assert docId >= 0 : " base=" + docBase + " doc=" + doc;
|
|
||||||
bits.set(docId);
|
|
||||||
lastDocCollected = docId;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void doSetNextReader(LeafReaderContext context) throws IOException {
|
|
||||||
docBase = context.docBase;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public ScoreMode scoreMode() {
|
|
||||||
return ScoreMode.COMPLETE;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
Loading…
Reference in New Issue