mirror of https://github.com/apache/lucene.git
LUCENE-10418: Optimize `Query#rewrite` in the non-scoring case. (#672)
This commit is contained in:
parent
86bd921fce
commit
8fb6543280
|
@ -35,7 +35,9 @@ Improvements
|
||||||
|
|
||||||
Optimizations
|
Optimizations
|
||||||
---------------------
|
---------------------
|
||||||
(No changes)
|
|
||||||
|
* LUCENE-10418: More `Query#rewrite` optimizations for the non-scoring case.
|
||||||
|
(Adrien Grand)
|
||||||
|
|
||||||
Bug Fixes
|
Bug Fixes
|
||||||
---------------------
|
---------------------
|
||||||
|
|
|
@ -191,38 +191,41 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
|
||||||
return clauses.iterator();
|
return clauses.iterator();
|
||||||
}
|
}
|
||||||
|
|
||||||
private BooleanQuery rewriteNoScoring() {
|
// Utility method for rewriting BooleanQuery when scores are not needed.
|
||||||
boolean keepShould =
|
// This is called from ConstantScoreQuery#rewrite
|
||||||
|
BooleanQuery rewriteNoScoring(IndexReader reader) throws IOException {
|
||||||
|
boolean actuallyRewritten = false;
|
||||||
|
BooleanQuery.Builder newQuery =
|
||||||
|
new BooleanQuery.Builder().setMinimumNumberShouldMatch(getMinimumNumberShouldMatch());
|
||||||
|
|
||||||
|
final boolean keepShould =
|
||||||
getMinimumNumberShouldMatch() > 0
|
getMinimumNumberShouldMatch() > 0
|
||||||
|| (clauseSets.get(Occur.MUST).size() + clauseSets.get(Occur.FILTER).size() == 0);
|
|| (clauseSets.get(Occur.MUST).size() + clauseSets.get(Occur.FILTER).size() == 0);
|
||||||
|
|
||||||
if (clauseSets.get(Occur.MUST).size() == 0 && keepShould) {
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
BooleanQuery.Builder newQuery = new BooleanQuery.Builder();
|
|
||||||
|
|
||||||
newQuery.setMinimumNumberShouldMatch(getMinimumNumberShouldMatch());
|
|
||||||
for (BooleanClause clause : clauses) {
|
for (BooleanClause clause : clauses) {
|
||||||
switch (clause.getOccur()) {
|
Query query = clause.getQuery();
|
||||||
case MUST:
|
Query rewritten = new ConstantScoreQuery(query).rewrite(reader);
|
||||||
{
|
if (rewritten instanceof ConstantScoreQuery) {
|
||||||
newQuery.add(clause.getQuery(), Occur.FILTER);
|
rewritten = ((ConstantScoreQuery) rewritten).getQuery();
|
||||||
break;
|
|
||||||
}
|
|
||||||
case SHOULD:
|
|
||||||
{
|
|
||||||
if (keepShould) {
|
|
||||||
newQuery.add(clause);
|
|
||||||
}
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
case FILTER:
|
|
||||||
case MUST_NOT:
|
|
||||||
default:
|
|
||||||
{
|
|
||||||
newQuery.add(clause);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
BooleanClause.Occur occur = clause.getOccur();
|
||||||
|
if (occur == Occur.SHOULD && keepShould == false) {
|
||||||
|
// ignore clause
|
||||||
|
actuallyRewritten = true;
|
||||||
|
} else if (occur == Occur.MUST) {
|
||||||
|
// replace MUST clauses with FILTER clauses
|
||||||
|
newQuery.add(rewritten, Occur.FILTER);
|
||||||
|
actuallyRewritten = true;
|
||||||
|
} else if (query != rewritten) {
|
||||||
|
newQuery.add(rewritten, occur);
|
||||||
|
actuallyRewritten = true;
|
||||||
|
} else {
|
||||||
|
newQuery.add(clause);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (actuallyRewritten == false) {
|
||||||
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
return newQuery.build();
|
return newQuery.build();
|
||||||
|
@ -231,11 +234,7 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
|
||||||
@Override
|
@Override
|
||||||
public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost)
|
public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
BooleanQuery query = this;
|
return new BooleanWeight(this, searcher, scoreMode, boost);
|
||||||
if (scoreMode.needsScores() == false) {
|
|
||||||
query = rewriteNoScoring();
|
|
||||||
}
|
|
||||||
return new BooleanWeight(query, searcher, scoreMode, boost);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -274,12 +273,22 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
|
||||||
boolean actuallyRewritten = false;
|
boolean actuallyRewritten = false;
|
||||||
for (BooleanClause clause : this) {
|
for (BooleanClause clause : this) {
|
||||||
Query query = clause.getQuery();
|
Query query = clause.getQuery();
|
||||||
Query rewritten = query.rewrite(reader);
|
BooleanClause.Occur occur = clause.getOccur();
|
||||||
|
Query rewritten;
|
||||||
|
if (occur == Occur.FILTER || occur == Occur.MUST_NOT) {
|
||||||
|
// Clauses that are not involved in scoring can get some extra simplifications
|
||||||
|
rewritten = new ConstantScoreQuery(query).rewrite(reader);
|
||||||
|
if (rewritten instanceof ConstantScoreQuery) {
|
||||||
|
rewritten = ((ConstantScoreQuery) rewritten).getQuery();
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
rewritten = query.rewrite(reader);
|
||||||
|
}
|
||||||
if (rewritten != query || query.getClass() == MatchNoDocsQuery.class) {
|
if (rewritten != query || query.getClass() == MatchNoDocsQuery.class) {
|
||||||
// rewrite clause
|
// rewrite clause
|
||||||
actuallyRewritten = true;
|
actuallyRewritten = true;
|
||||||
if (rewritten.getClass() == MatchNoDocsQuery.class) {
|
if (rewritten.getClass() == MatchNoDocsQuery.class) {
|
||||||
switch (clause.getOccur()) {
|
switch (occur) {
|
||||||
case SHOULD:
|
case SHOULD:
|
||||||
case MUST_NOT:
|
case MUST_NOT:
|
||||||
// the clause can be safely ignored
|
// the clause can be safely ignored
|
||||||
|
@ -289,7 +298,7 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
|
||||||
return rewritten;
|
return rewritten;
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
builder.add(rewritten, clause.getOccur());
|
builder.add(rewritten, occur);
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
// leave as-is
|
// leave as-is
|
||||||
|
|
|
@ -411,6 +411,13 @@ final class BooleanWeight extends Weight {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if (scoreMode.needsScores() == false
|
||||||
|
&& minShouldMatch == 0
|
||||||
|
&& scorers.get(Occur.MUST).size() + scorers.get(Occur.FILTER).size() > 0) {
|
||||||
|
// Purely optional clauses are useless without scoring.
|
||||||
|
scorers.get(Occur.SHOULD).clear();
|
||||||
|
}
|
||||||
|
|
||||||
return new Boolean2ScorerSupplier(this, scorers, scoreMode, minShouldMatch);
|
return new Boolean2ScorerSupplier(this, scorers, scoreMode, minShouldMatch);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -43,6 +43,16 @@ public final class ConstantScoreQuery extends Query {
|
||||||
public Query rewrite(IndexReader reader) throws IOException {
|
public Query rewrite(IndexReader reader) throws IOException {
|
||||||
Query rewritten = query.rewrite(reader);
|
Query rewritten = query.rewrite(reader);
|
||||||
|
|
||||||
|
// Do some extra simplifications that are legal since scores are not needed on the wrapped
|
||||||
|
// query.
|
||||||
|
if (rewritten instanceof BoostQuery) {
|
||||||
|
rewritten = ((BoostQuery) rewritten).getQuery();
|
||||||
|
} else if (rewritten instanceof ConstantScoreQuery) {
|
||||||
|
rewritten = ((ConstantScoreQuery) rewritten).getQuery();
|
||||||
|
} else if (rewritten instanceof BooleanQuery) {
|
||||||
|
rewritten = ((BooleanQuery) rewritten).rewriteNoScoring(reader);
|
||||||
|
}
|
||||||
|
|
||||||
if (rewritten.getClass() == MatchNoDocsQuery.class) {
|
if (rewritten.getClass() == MatchNoDocsQuery.class) {
|
||||||
// bubble up MatchNoDocsQuery
|
// bubble up MatchNoDocsQuery
|
||||||
return rewritten;
|
return rewritten;
|
||||||
|
|
|
@ -446,7 +446,7 @@ public class IndexSearcher {
|
||||||
* possible.
|
* possible.
|
||||||
*/
|
*/
|
||||||
public int count(Query query) throws IOException {
|
public int count(Query query) throws IOException {
|
||||||
query = rewrite(query);
|
query = rewrite(query, false);
|
||||||
final Weight weight = createWeight(query, ScoreMode.COMPLETE_NO_SCORES, 1);
|
final Weight weight = createWeight(query, ScoreMode.COMPLETE_NO_SCORES, 1);
|
||||||
|
|
||||||
final CollectorManager<ShortcutHitCountCollector, Integer> shortcutCollectorManager =
|
final CollectorManager<ShortcutHitCountCollector, Integer> shortcutCollectorManager =
|
||||||
|
@ -551,7 +551,7 @@ public class IndexSearcher {
|
||||||
* clauses.
|
* clauses.
|
||||||
*/
|
*/
|
||||||
public void search(Query query, Collector results) throws IOException {
|
public void search(Query query, Collector results) throws IOException {
|
||||||
query = rewrite(query);
|
query = rewrite(query, results.scoreMode().needsScores());
|
||||||
search(leafContexts, createWeight(query, results.scoreMode(), 1), results);
|
search(leafContexts, createWeight(query, results.scoreMode(), 1), results);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -682,7 +682,7 @@ public class IndexSearcher {
|
||||||
public <C extends Collector, T> T search(Query query, CollectorManager<C, T> collectorManager)
|
public <C extends Collector, T> T search(Query query, CollectorManager<C, T> collectorManager)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
final C firstCollector = collectorManager.newCollector();
|
final C firstCollector = collectorManager.newCollector();
|
||||||
query = rewrite(query);
|
query = rewrite(query, firstCollector.scoreMode().needsScores());
|
||||||
final Weight weight = createWeight(query, firstCollector.scoreMode(), 1);
|
final Weight weight = createWeight(query, firstCollector.scoreMode(), 1);
|
||||||
return search(weight, collectorManager, firstCollector);
|
return search(weight, collectorManager, firstCollector);
|
||||||
}
|
}
|
||||||
|
@ -795,6 +795,15 @@ public class IndexSearcher {
|
||||||
return query;
|
return query;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private Query rewrite(Query original, boolean needsScores) throws IOException {
|
||||||
|
if (needsScores) {
|
||||||
|
return rewrite(original);
|
||||||
|
} else {
|
||||||
|
// Take advantage of the few extra rewrite rules of ConstantScoreQuery.
|
||||||
|
return rewrite(new ConstantScoreQuery(original));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns a QueryVisitor which recursively checks the total number of clauses that a query and
|
* Returns a QueryVisitor which recursively checks the total number of clauses that a query and
|
||||||
* its children cumulatively have and validates that the total number does not exceed the
|
* its children cumulatively have and validates that the total number does not exceed the
|
||||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.lucene.search;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.Random;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
import org.apache.lucene.document.Document;
|
import org.apache.lucene.document.Document;
|
||||||
|
@ -405,7 +406,7 @@ public class TestBooleanRewrites extends LuceneTestCase {
|
||||||
|
|
||||||
final int iters = atLeast(1000);
|
final int iters = atLeast(1000);
|
||||||
for (int i = 0; i < iters; ++i) {
|
for (int i = 0; i < iters; ++i) {
|
||||||
Query query = randomQuery();
|
Query query = randomBooleanQuery(random());
|
||||||
final TopDocs td1 = searcher1.search(query, 100);
|
final TopDocs td1 = searcher1.search(query, 100);
|
||||||
final TopDocs td2 = searcher2.search(query, 100);
|
final TopDocs td2 = searcher2.search(query, 100);
|
||||||
assertEquals(td1, td2);
|
assertEquals(td1, td2);
|
||||||
|
@ -415,29 +416,41 @@ public class TestBooleanRewrites extends LuceneTestCase {
|
||||||
dir.close();
|
dir.close();
|
||||||
}
|
}
|
||||||
|
|
||||||
private Query randomBooleanQuery() {
|
private Query randomBooleanQuery(Random random) {
|
||||||
if (random().nextInt(10) == 0) {
|
final int numClauses = random.nextInt(5);
|
||||||
return new BoostQuery(randomBooleanQuery(), TestUtil.nextInt(random(), 1, 10));
|
|
||||||
}
|
|
||||||
final int numClauses = random().nextInt(5);
|
|
||||||
BooleanQuery.Builder b = new BooleanQuery.Builder();
|
BooleanQuery.Builder b = new BooleanQuery.Builder();
|
||||||
int numShoulds = 0;
|
int numShoulds = 0;
|
||||||
for (int i = 0; i < numClauses; ++i) {
|
for (int i = 0; i < numClauses; ++i) {
|
||||||
final Occur occur = Occur.values()[random().nextInt(Occur.values().length)];
|
final Occur occur = Occur.values()[random.nextInt(Occur.values().length)];
|
||||||
if (occur == Occur.SHOULD) {
|
if (occur == Occur.SHOULD) {
|
||||||
numShoulds++;
|
numShoulds++;
|
||||||
}
|
}
|
||||||
final Query query = randomQuery();
|
final Query query = randomQuery(random);
|
||||||
b.add(query, occur);
|
b.add(query, occur);
|
||||||
}
|
}
|
||||||
b.setMinimumNumberShouldMatch(
|
b.setMinimumNumberShouldMatch(
|
||||||
random().nextBoolean() ? 0 : TestUtil.nextInt(random(), 0, numShoulds + 1));
|
random().nextBoolean() ? 0 : TestUtil.nextInt(random(), 0, numShoulds + 1));
|
||||||
return b.build();
|
Query query = b.build();
|
||||||
|
if (random.nextBoolean()) {
|
||||||
|
query = randomWrapper(random, query);
|
||||||
|
}
|
||||||
|
return query;
|
||||||
}
|
}
|
||||||
|
|
||||||
private Query randomQuery() {
|
private Query randomWrapper(Random random, Query query) {
|
||||||
if (random().nextInt(10) == 0) {
|
switch (random.nextInt(2)) {
|
||||||
return new BoostQuery(randomBooleanQuery(), TestUtil.nextInt(random(), 1, 10));
|
case 0:
|
||||||
|
return new BoostQuery(query, TestUtil.nextInt(random, 0, 4));
|
||||||
|
case 1:
|
||||||
|
return new ConstantScoreQuery(query);
|
||||||
|
default:
|
||||||
|
throw new AssertionError();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private Query randomQuery(Random random) {
|
||||||
|
if (random.nextInt(5) == 0) {
|
||||||
|
return randomWrapper(random, randomQuery(random));
|
||||||
}
|
}
|
||||||
switch (random().nextInt(6)) {
|
switch (random().nextInt(6)) {
|
||||||
case 0:
|
case 0:
|
||||||
|
@ -451,7 +464,7 @@ public class TestBooleanRewrites extends LuceneTestCase {
|
||||||
case 4:
|
case 4:
|
||||||
return new TermQuery(new Term("body", "d"));
|
return new TermQuery(new Term("body", "d"));
|
||||||
case 5:
|
case 5:
|
||||||
return randomBooleanQuery();
|
return randomBooleanQuery(random);
|
||||||
default:
|
default:
|
||||||
throw new AssertionError();
|
throw new AssertionError();
|
||||||
}
|
}
|
||||||
|
@ -609,59 +622,57 @@ public class TestBooleanRewrites extends LuceneTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testDiscardShouldClauses() throws IOException {
|
public void testDiscardShouldClauses() throws IOException {
|
||||||
Directory dir = newDirectory();
|
IndexSearcher searcher = newSearcher(new MultiReader());
|
||||||
RandomIndexWriter w = new RandomIndexWriter(random(), dir);
|
|
||||||
Document doc = new Document();
|
|
||||||
Field f = newTextField("field", "a", Field.Store.NO);
|
|
||||||
doc.add(f);
|
|
||||||
w.addDocument(doc);
|
|
||||||
w.commit();
|
|
||||||
|
|
||||||
DirectoryReader reader = w.getReader();
|
Query query1 =
|
||||||
final IndexSearcher searcher = new IndexSearcher(reader);
|
new ConstantScoreQuery(
|
||||||
|
new BooleanQuery.Builder()
|
||||||
|
.add(new TermQuery(new Term("field", "a")), Occur.MUST)
|
||||||
|
.add(new TermQuery(new Term("field", "b")), Occur.SHOULD)
|
||||||
|
.build());
|
||||||
|
Query rewritten1 = new ConstantScoreQuery(new TermQuery(new Term("field", "a")));
|
||||||
|
assertEquals(rewritten1, searcher.rewrite(query1));
|
||||||
|
|
||||||
BooleanQuery.Builder query1 = new BooleanQuery.Builder();
|
Query query2 =
|
||||||
query1.add(new TermQuery(new Term("field", "a")), Occur.MUST);
|
new ConstantScoreQuery(
|
||||||
query1.add(new TermQuery(new Term("field", "b")), Occur.SHOULD);
|
new BooleanQuery.Builder()
|
||||||
|
.add(new TermQuery(new Term("field", "a")), Occur.MUST)
|
||||||
|
.add(new TermQuery(new Term("field", "b")), Occur.SHOULD)
|
||||||
|
.add(new TermQuery(new Term("field", "c")), Occur.FILTER)
|
||||||
|
.build());
|
||||||
|
Query rewritten2 =
|
||||||
|
new ConstantScoreQuery(
|
||||||
|
new BooleanQuery.Builder()
|
||||||
|
.add(new TermQuery(new Term("field", "a")), Occur.FILTER)
|
||||||
|
.add(new TermQuery(new Term("field", "c")), Occur.FILTER)
|
||||||
|
.build());
|
||||||
|
assertEquals(rewritten2, searcher.rewrite(query2));
|
||||||
|
|
||||||
query1.setMinimumNumberShouldMatch(0);
|
Query query3 =
|
||||||
|
new ConstantScoreQuery(
|
||||||
|
new BooleanQuery.Builder()
|
||||||
|
.add(new TermQuery(new Term("field", "a")), Occur.SHOULD)
|
||||||
|
.add(new TermQuery(new Term("field", "b")), Occur.SHOULD)
|
||||||
|
.build());
|
||||||
|
assertSame(query3, searcher.rewrite(query3));
|
||||||
|
|
||||||
Weight weight =
|
Query query4 =
|
||||||
searcher.createWeight(searcher.rewrite(query1.build()), ScoreMode.COMPLETE_NO_SCORES, 1);
|
new ConstantScoreQuery(
|
||||||
|
new BooleanQuery.Builder()
|
||||||
|
.add(new TermQuery(new Term("field", "a")), Occur.SHOULD)
|
||||||
|
.add(new TermQuery(new Term("field", "b")), Occur.MUST_NOT)
|
||||||
|
.build());
|
||||||
|
assertSame(query4, searcher.rewrite(query4));
|
||||||
|
|
||||||
Query rewrittenQuery1 = weight.getQuery();
|
Query query5 =
|
||||||
|
new ConstantScoreQuery(
|
||||||
assertTrue(rewrittenQuery1 instanceof BooleanQuery);
|
new BooleanQuery.Builder()
|
||||||
|
.setMinimumNumberShouldMatch(1)
|
||||||
BooleanQuery booleanRewrittenQuery1 = (BooleanQuery) rewrittenQuery1;
|
.add(new TermQuery(new Term("field", "a")), Occur.SHOULD)
|
||||||
|
.add(new TermQuery(new Term("field", "b")), Occur.SHOULD)
|
||||||
for (BooleanClause clause : booleanRewrittenQuery1.clauses()) {
|
.add(new TermQuery(new Term("field", "c")), Occur.FILTER)
|
||||||
assertNotEquals(clause.getOccur(), Occur.SHOULD);
|
.build());
|
||||||
}
|
assertSame(query5, searcher.rewrite(query5));
|
||||||
|
|
||||||
BooleanQuery.Builder query2 = new BooleanQuery.Builder();
|
|
||||||
query2.add(new TermQuery(new Term("field", "a")), Occur.MUST);
|
|
||||||
query2.add(new TermQuery(new Term("field", "b")), Occur.SHOULD);
|
|
||||||
query2.add(new TermQuery(new Term("field", "c")), Occur.FILTER);
|
|
||||||
|
|
||||||
query2.setMinimumNumberShouldMatch(0);
|
|
||||||
|
|
||||||
weight =
|
|
||||||
searcher.createWeight(searcher.rewrite(query2.build()), ScoreMode.COMPLETE_NO_SCORES, 1);
|
|
||||||
|
|
||||||
Query rewrittenQuery2 = weight.getQuery();
|
|
||||||
|
|
||||||
assertTrue(rewrittenQuery2 instanceof BooleanQuery);
|
|
||||||
|
|
||||||
BooleanQuery booleanRewrittenQuery2 = (BooleanQuery) rewrittenQuery1;
|
|
||||||
|
|
||||||
for (BooleanClause clause : booleanRewrittenQuery2.clauses()) {
|
|
||||||
assertNotEquals(clause.getOccur(), Occur.SHOULD);
|
|
||||||
}
|
|
||||||
|
|
||||||
reader.close();
|
|
||||||
w.close();
|
|
||||||
dir.close();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testShouldMatchNoDocsQuery() throws IOException {
|
public void testShouldMatchNoDocsQuery() throws IOException {
|
||||||
|
@ -713,4 +724,63 @@ public class TestBooleanRewrites extends LuceneTestCase {
|
||||||
BooleanQuery query = new BooleanQuery.Builder().build();
|
BooleanQuery query = new BooleanQuery.Builder().build();
|
||||||
assertEquals(new MatchNoDocsQuery(), searcher.rewrite(query));
|
assertEquals(new MatchNoDocsQuery(), searcher.rewrite(query));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void testSimplifyFilterClauses() throws IOException {
|
||||||
|
IndexSearcher searcher = newSearcher(new MultiReader());
|
||||||
|
|
||||||
|
BooleanQuery query1 =
|
||||||
|
new BooleanQuery.Builder()
|
||||||
|
.add(new TermQuery(new Term("foo", "bar")), Occur.MUST)
|
||||||
|
.add(new ConstantScoreQuery(new TermQuery(new Term("foo", "baz"))), Occur.FILTER)
|
||||||
|
.build();
|
||||||
|
BooleanQuery expected1 =
|
||||||
|
new BooleanQuery.Builder()
|
||||||
|
.add(new TermQuery(new Term("foo", "bar")), Occur.MUST)
|
||||||
|
.add(new TermQuery(new Term("foo", "baz")), Occur.FILTER)
|
||||||
|
.build();
|
||||||
|
assertEquals(expected1, searcher.rewrite(query1));
|
||||||
|
|
||||||
|
BooleanQuery query2 =
|
||||||
|
new BooleanQuery.Builder()
|
||||||
|
.add(new TermQuery(new Term("foo", "bar")), Occur.FILTER)
|
||||||
|
.add(new ConstantScoreQuery(new TermQuery(new Term("foo", "bar"))), Occur.FILTER)
|
||||||
|
.build();
|
||||||
|
Query expected2 =
|
||||||
|
new BoostQuery(new ConstantScoreQuery(new TermQuery(new Term("foo", "bar"))), 0);
|
||||||
|
assertEquals(expected2, searcher.rewrite(query2));
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testSimplifyMustNotClauses() throws IOException {
|
||||||
|
IndexSearcher searcher = newSearcher(new MultiReader());
|
||||||
|
|
||||||
|
BooleanQuery query =
|
||||||
|
new BooleanQuery.Builder()
|
||||||
|
.add(new TermQuery(new Term("foo", "bar")), Occur.MUST)
|
||||||
|
.add(new ConstantScoreQuery(new TermQuery(new Term("foo", "baz"))), Occur.MUST_NOT)
|
||||||
|
.build();
|
||||||
|
BooleanQuery expected =
|
||||||
|
new BooleanQuery.Builder()
|
||||||
|
.add(new TermQuery(new Term("foo", "bar")), Occur.MUST)
|
||||||
|
.add(new TermQuery(new Term("foo", "baz")), Occur.MUST_NOT)
|
||||||
|
.build();
|
||||||
|
assertEquals(expected, searcher.rewrite(query));
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testSimplifyNonScoringShouldClauses() throws IOException {
|
||||||
|
IndexSearcher searcher = newSearcher(new MultiReader());
|
||||||
|
|
||||||
|
Query query =
|
||||||
|
new ConstantScoreQuery(
|
||||||
|
new BooleanQuery.Builder()
|
||||||
|
.add(new TermQuery(new Term("foo", "bar")), Occur.SHOULD)
|
||||||
|
.add(new ConstantScoreQuery(new TermQuery(new Term("foo", "baz"))), Occur.SHOULD)
|
||||||
|
.build());
|
||||||
|
Query expected =
|
||||||
|
new ConstantScoreQuery(
|
||||||
|
new BooleanQuery.Builder()
|
||||||
|
.add(new TermQuery(new Term("foo", "bar")), Occur.SHOULD)
|
||||||
|
.add(new TermQuery(new Term("foo", "baz")), Occur.SHOULD)
|
||||||
|
.build());
|
||||||
|
assertEquals(expected, searcher.rewrite(query));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -91,5 +91,8 @@ public class TestBoostQuery extends LuceneTestCase {
|
||||||
|
|
||||||
Query query = new BoostQuery(new MatchNoDocsQuery(), 2f);
|
Query query = new BoostQuery(new MatchNoDocsQuery(), 2f);
|
||||||
assertEquals(new MatchNoDocsQuery(), searcher.rewrite(query));
|
assertEquals(new MatchNoDocsQuery(), searcher.rewrite(query));
|
||||||
|
|
||||||
|
query = new BoostQuery(new MatchNoDocsQuery(), 0f);
|
||||||
|
assertEquals(new MatchNoDocsQuery(), searcher.rewrite(query));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -1971,7 +1971,19 @@ public class TestLRUQueryCache extends LuceneTestCase {
|
||||||
w.addDocuments(Arrays.asList(doc1, doc2, doc3));
|
w.addDocuments(Arrays.asList(doc1, doc2, doc3));
|
||||||
final IndexReader reader = w.getReader();
|
final IndexReader reader = w.getReader();
|
||||||
final IndexSearcher searcher = newSearcher(reader);
|
final IndexSearcher searcher = newSearcher(reader);
|
||||||
final UsageTrackingQueryCachingPolicy policy = new UsageTrackingQueryCachingPolicy();
|
final QueryCachingPolicy policy =
|
||||||
|
new QueryCachingPolicy() {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean shouldCache(Query query) throws IOException {
|
||||||
|
return query.getClass() != TermQuery.class;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void onUse(Query query) {
|
||||||
|
// no-op
|
||||||
|
}
|
||||||
|
};
|
||||||
searcher.setQueryCachingPolicy(policy);
|
searcher.setQueryCachingPolicy(policy);
|
||||||
w.close();
|
w.close();
|
||||||
|
|
||||||
|
|
|
@ -46,6 +46,9 @@ public class TestNeedsScores extends LuceneTestCase {
|
||||||
}
|
}
|
||||||
reader = iw.getReader();
|
reader = iw.getReader();
|
||||||
searcher = newSearcher(reader);
|
searcher = newSearcher(reader);
|
||||||
|
// Needed so that the cache doesn't consume weights with ScoreMode.COMPLETE_NO_SCORES for the
|
||||||
|
// purpose of populating the cache.
|
||||||
|
searcher.setQueryCache(null);
|
||||||
iw.close();
|
iw.close();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue