Delete TimeLimitingCollector after deprecation (#13243)

Co-authored-by: Kaival Parikh <kaivalp2000@gmail.com>
This commit is contained in:
Kaival Parikh 2024-03-29 20:08:44 +05:30 committed by GitHub
parent ce978d7646
commit d6eb126871
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
3 changed files with 4 additions and 732 deletions

View File

@ -192,6 +192,10 @@ enum.
`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
### Test framework package migration and module (LUCENE-10301)

View File

@ -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.
}
}
}

View File

@ -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;
}
}
}