You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jp...@apache.org on 2022/07/04 15:32:47 UTC
[lucene] branch main updated: LUCENE-10151: Some fixes to query timeouts. (#996)
This is an automated email from the ASF dual-hosted git repository.
jpountz pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/lucene.git
The following commit(s) were added to refs/heads/main by this push:
new 81d4a7a69f1 LUCENE-10151: Some fixes to query timeouts. (#996)
81d4a7a69f1 is described below
commit 81d4a7a69f1c9085e40df412be87de22d0aa8cd6
Author: Adrien Grand <jp...@gmail.com>
AuthorDate: Mon Jul 4 17:32:38 2022 +0200
LUCENE-10151: Some fixes to query timeouts. (#996)
I noticed some minor bugs in the original PR #927 that this PR should fix:
- When a timeout is set, we would no longer catch
`CollectionTerminatedException`.
- I added randomization to `LuceneTestCase` to randomly set a timeout, it
would have caught the above bug.
- Fixed visibility of `TimeLimitingBulkScorer`.
---
.../org/apache/lucene/search/IndexSearcher.java | 45 ++++++++++++----------
.../lucene/search/TimeLimitingBulkScorer.java | 8 ++--
.../org/apache/lucene/facet/TestDrillSideways.java | 6 ++-
.../lucene/facet/range/TestRangeFacetCounts.java | 6 +++
.../apache/lucene/tests/util/LuceneTestCase.java | 10 +++++
5 files changed, 50 insertions(+), 25 deletions(-)
diff --git a/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java b/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
index 42d99d878d8..e04b2566d9b 100644
--- a/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
+++ b/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
@@ -85,7 +85,11 @@ public class IndexSearcher {
private static QueryCache DEFAULT_QUERY_CACHE;
private static QueryCachingPolicy DEFAULT_CACHING_POLICY = new UsageTrackingQueryCachingPolicy();
private QueryTimeout queryTimeout = null;
- private boolean partialResult = false;
+ // partialResult may be set on one of the threads of the executor. It may be correct to not make
+ // this variable volatile since joining these threads should ensure a happens-before relationship
+ // that guarantees that writes become visible on the main thread, but making the variable volatile
+ // shouldn't hurt either.
+ private volatile boolean partialResult = false;
static {
final int maxCachedQueries = 1000;
@@ -487,7 +491,8 @@ public class IndexSearcher {
return search(query, manager);
}
- public void setTimeout(QueryTimeout queryTimeout) throws IOException {
+ /** Set a {@link QueryTimeout} for all searches that run through this {@link IndexSearcher}. */
+ public void setTimeout(QueryTimeout queryTimeout) {
this.queryTimeout = queryTimeout;
}
@@ -514,9 +519,11 @@ public class IndexSearcher {
search(leafContexts, createWeight(query, results.scoreMode(), 1), results);
}
+ /** Returns true if any search hit the {@link #setTimeout(QueryTimeout) timeout}. */
public boolean timedOut() {
return partialResult;
}
+
/**
* Search implementation with arbitrary sorting, plus control over whether hit scores and max
* score should be computed. Finds the top <code>n</code> hits for <code>query</code>, and sorting
@@ -730,29 +737,25 @@ public class IndexSearcher {
}
BulkScorer scorer = weight.bulkScorer(ctx);
if (scorer != null) {
- if (queryTimeout != null) {
- TimeLimitingBulkScorer timeLimitingBulkScorer =
- new TimeLimitingBulkScorer(scorer, queryTimeout);
- try {
- timeLimitingBulkScorer.score(leafCollector, ctx.reader().getLiveDocs());
- } catch (
- @SuppressWarnings("unused")
- TimeLimitingBulkScorer.TimeExceededException e) {
- partialResult = true;
- }
- } else {
- try {
- scorer.score(leafCollector, ctx.reader().getLiveDocs());
- } catch (
- @SuppressWarnings("unused")
- CollectionTerminatedException e) {
- // collection was terminated prematurely
- // continue with the following leaf
- }
+ if (queryTimeout != null && queryTimeout.isTimeoutEnabled()) {
+ scorer = new TimeLimitingBulkScorer(scorer, queryTimeout);
+ }
+ try {
+ scorer.score(leafCollector, ctx.reader().getLiveDocs());
+ } catch (
+ @SuppressWarnings("unused")
+ CollectionTerminatedException e) {
+ // collection was terminated prematurely
+ // continue with the following leaf
+ } catch (
+ @SuppressWarnings("unused")
+ TimeLimitingBulkScorer.TimeExceededException e) {
+ partialResult = true;
}
}
}
}
+
/**
* Expert: called to re-write queries into primitive queries.
*
diff --git a/lucene/core/src/java/org/apache/lucene/search/TimeLimitingBulkScorer.java b/lucene/core/src/java/org/apache/lucene/search/TimeLimitingBulkScorer.java
index 4f17d40dc7c..4d8c00b04cd 100644
--- a/lucene/core/src/java/org/apache/lucene/search/TimeLimitingBulkScorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/TimeLimitingBulkScorer.java
@@ -28,10 +28,11 @@ import org.apache.lucene.util.Bits;
*
* @see org.apache.lucene.index.ExitableDirectoryReader
*/
-public class TimeLimitingBulkScorer extends BulkScorer {
+final class TimeLimitingBulkScorer extends BulkScorer {
// We score chunks of documents at a time so as to avoid the cost of checking the timeout for
// every document we score.
static final int INTERVAL = 100;
+
/** Thrown when elapsed search time exceeds allowed search time. */
@SuppressWarnings("serial")
static class TimeExceededException extends RuntimeException {
@@ -41,8 +42,9 @@ public class TimeLimitingBulkScorer extends BulkScorer {
}
}
- private BulkScorer in;
- private QueryTimeout queryTimeout;
+ private final BulkScorer in;
+ private final QueryTimeout queryTimeout;
+
/**
* Create a TimeLimitingBulkScorer wrapper over another {@link BulkScorer} with a specified
* timeout.
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/TestDrillSideways.java b/lucene/facet/src/test/org/apache/lucene/facet/TestDrillSideways.java
index e68be0f5975..2a6f15ad23b 100644
--- a/lucene/facet/src/test/org/apache/lucene/facet/TestDrillSideways.java
+++ b/lucene/facet/src/test/org/apache/lucene/facet/TestDrillSideways.java
@@ -132,7 +132,11 @@ public class TestDrillSideways extends FacetTestCase {
private IndexSearcher getNewSearcher(IndexReader reader) {
// Do not wrap with an asserting searcher, since DrillSidewaysQuery doesn't
// implement all the required components like Weight#scorer.
- return newSearcher(reader, true, false, random().nextBoolean());
+ IndexSearcher searcher = newSearcher(reader, true, false, random().nextBoolean());
+ // DrillSideways requires the entire range of docs to be scored at once, so it doesn't support
+ // timeouts whose implementation scores one window of doc IDs at a time.
+ searcher.setTimeout(null);
+ return searcher;
}
// See LUCENE-10060:
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java b/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java
index e33556337b4..1ce7642cfd1 100644
--- a/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java
+++ b/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java
@@ -458,6 +458,9 @@ public class TestRangeFacetCounts extends FacetTestCase {
final TaxonomyReader tr = new DirectoryTaxonomyReader(tw);
IndexSearcher s = newSearcher(r, false, false);
+ // DrillSideways requires the entire range of docs to be scored at once, so it doesn't support
+ // timeouts whose implementation scores one window of doc IDs at a time.
+ s.setTimeout(null);
if (VERBOSE) {
System.out.println("TEST: searcher=" + s);
@@ -1555,6 +1558,9 @@ public class TestRangeFacetCounts extends FacetTestCase {
IndexReader r = writer.getReader();
IndexSearcher s = newSearcher(r, false, false);
+ // DrillSideways requires the entire range of docs to be scored at once, so it doesn't support
+ // timeouts whose implementation scores one window of doc IDs at a time.
+ s.setTimeout(null);
FacetsCollector fc = s.search(new MatchAllDocsQuery(), new FacetsCollectorManager());
final DoubleRange[] ranges =
diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/util/LuceneTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/tests/util/LuceneTestCase.java
index 09086b69995..2c1a2ebc527 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/tests/util/LuceneTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/tests/util/LuceneTestCase.java
@@ -138,6 +138,7 @@ import org.apache.lucene.index.ParallelCompositeReader;
import org.apache.lucene.index.ParallelLeafReader;
import org.apache.lucene.index.PointValues;
import org.apache.lucene.index.PostingsEnum;
+import org.apache.lucene.index.QueryTimeout;
import org.apache.lucene.index.SerialMergeScheduler;
import org.apache.lucene.index.SimpleMergedSegmentWarmer;
import org.apache.lucene.index.SortedDocValues;
@@ -1993,6 +1994,15 @@ public abstract class LuceneTestCase extends Assert {
}
ret.setSimilarity(classEnvRule.similarity);
ret.setQueryCachingPolicy(MAYBE_CACHE_POLICY);
+ if (random().nextBoolean()) {
+ ret.setTimeout(
+ new QueryTimeout() {
+ @Override
+ public boolean shouldExit() {
+ return false;
+ }
+ });
+ }
return ret;
}
}