mirror of https://github.com/apache/lucene.git
LUCENE-6227: Add BooleanClause.Occur.FILTER.
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1658764 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
0b4cdec6ac
commit
a70006f2f5
|
@ -38,6 +38,9 @@ New Features
|
|||
|
||||
* LUCENE-6191: New spatial 2D heatmap faceting for PrefixTreeStrategy. (David Smiley)
|
||||
|
||||
* LUCENE-6227: Added BooleanClause.Occur.FILTER to filter documents without
|
||||
participating in scoring (on the contrary to MUST). (Adrien Grand)
|
||||
|
||||
Bug Fixes
|
||||
|
||||
* LUCENE-6190: Spatial pointsOnly flag on PrefixTreeStrategy shouldn't switch all predicates to
|
||||
|
|
|
@ -26,6 +26,9 @@ public class BooleanClause {
|
|||
/** Use this operator for clauses that <i>must</i> appear in the matching documents. */
|
||||
MUST { @Override public String toString() { return "+"; } },
|
||||
|
||||
/** Like {@link #MUST} except that these clauses do not participate in scoring. */
|
||||
FILTER { @Override public String toString() { return "#"; } },
|
||||
|
||||
/** Use this operator for clauses that <i>should</i> appear in the
|
||||
* matching documents. For a BooleanQuery with no <code>MUST</code>
|
||||
* clauses one or more <code>SHOULD</code> clauses must match a document
|
||||
|
@ -36,7 +39,8 @@ public class BooleanClause {
|
|||
|
||||
/** Use this operator for clauses that <i>must not</i> appear in the matching documents.
|
||||
* Note that it is not possible to search for queries that only consist
|
||||
* of a <code>MUST_NOT</code> clause. */
|
||||
* of a <code>MUST_NOT</code> clause. These clauses do not contribute to the
|
||||
* score of documents. */
|
||||
MUST_NOT { @Override public String toString() { return "-"; } };
|
||||
|
||||
}
|
||||
|
@ -78,10 +82,12 @@ public class BooleanClause {
|
|||
}
|
||||
|
||||
public boolean isRequired() {
|
||||
return Occur.MUST == occur;
|
||||
return occur == Occur.MUST || occur == Occur.FILTER;
|
||||
}
|
||||
|
||||
|
||||
public boolean isScoring() {
|
||||
return occur == Occur.MUST || occur == Occur.SHOULD;
|
||||
}
|
||||
|
||||
/** Returns true if <code>o</code> is equal to this. */
|
||||
@Override
|
||||
|
@ -96,7 +102,7 @@ public class BooleanClause {
|
|||
/** Returns a hash code value for this object.*/
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return query.hashCode() ^ (Occur.MUST == occur?1:0) ^ (Occur.MUST_NOT == occur?2:0);
|
||||
return 31 * query.hashCode() + occur.hashCode();
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -19,17 +19,13 @@ package org.apache.lucene.search;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.search.BooleanClause.Occur;
|
||||
import org.apache.lucene.search.similarities.Similarity;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.ToStringUtils;
|
||||
|
||||
/** A Query that matches documents matching boolean combinations of other
|
||||
|
@ -175,14 +171,22 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
|
|||
|
||||
Query query = c.getQuery().rewrite(reader); // rewrite first
|
||||
|
||||
if (getBoost() != 1.0f) { // incorporate boost
|
||||
if (query == c.getQuery()) { // if rewrite was no-op
|
||||
query = query.clone(); // then clone before boost
|
||||
if (c.isScoring()) {
|
||||
if (getBoost() != 1.0f) { // incorporate boost
|
||||
if (query == c.getQuery()) { // if rewrite was no-op
|
||||
query = query.clone(); // then clone before boost
|
||||
}
|
||||
// Since the BooleanQuery only has 1 clause, the BooleanQuery will be
|
||||
// written out. Therefore the rewritten Query's boost must incorporate both
|
||||
// the clause's boost, and the boost of the BooleanQuery itself
|
||||
query.setBoost(getBoost() * query.getBoost());
|
||||
}
|
||||
} else {
|
||||
// our single clause is a filter
|
||||
if (query.getBoost() != 0f) {
|
||||
query = query.clone();
|
||||
query.setBoost(0);
|
||||
}
|
||||
// Since the BooleanQuery only has 1 clause, the BooleanQuery will be
|
||||
// written out. Therefore the rewritten Query's boost must incorporate both
|
||||
// the clause's boost, and the boost of the BooleanQuery itself
|
||||
query.setBoost(getBoost() * query.getBoost());
|
||||
}
|
||||
|
||||
return query;
|
||||
|
@ -214,7 +218,7 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
|
|||
@Override
|
||||
public void extractTerms(Set<Term> terms) {
|
||||
for (BooleanClause clause : clauses) {
|
||||
if (clause.getOccur() != Occur.MUST_NOT) {
|
||||
if (clause.isProhibited() == false) {
|
||||
clause.getQuery().extractTerms(terms);
|
||||
}
|
||||
}
|
||||
|
@ -223,7 +227,7 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
|
|||
@Override @SuppressWarnings("unchecked")
|
||||
public BooleanQuery clone() {
|
||||
BooleanQuery clone = (BooleanQuery)super.clone();
|
||||
clone.clauses = (ArrayList<BooleanClause>) this.clauses.clone();
|
||||
clone.clauses = new ArrayList<>(clauses);
|
||||
return clone;
|
||||
}
|
||||
|
||||
|
|
|
@ -18,11 +18,10 @@ package org.apache.lucene.search;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
|
||||
import org.apache.lucene.search.Scorer.ChildScorer;
|
||||
|
||||
/** Internal document-at-a-time scorers used to deal with stupid coord() computation */
|
||||
class BooleanTopLevelScorers {
|
||||
|
||||
|
@ -32,7 +31,7 @@ class BooleanTopLevelScorers {
|
|||
* to factor in coord().
|
||||
*/
|
||||
static class BoostedScorer extends FilterScorer {
|
||||
private final float boost;
|
||||
final float boost;
|
||||
|
||||
BoostedScorer(Scorer in, float boost) {
|
||||
super(in);
|
||||
|
@ -62,7 +61,7 @@ class BooleanTopLevelScorers {
|
|||
private final Scorer opt;
|
||||
|
||||
CoordinatingConjunctionScorer(Weight weight, float coords[], Scorer req, int reqCount, Scorer opt) {
|
||||
super(weight, new Scorer[] { req, opt });
|
||||
super(weight, Arrays.asList(req, opt), Arrays.asList(req, opt));
|
||||
this.coords = coords;
|
||||
this.req = req;
|
||||
this.reqCount = reqCount;
|
||||
|
|
|
@ -52,10 +52,9 @@ public class BooleanWeight extends Weight {
|
|||
weights = new ArrayList<>(query.clauses().size());
|
||||
for (int i = 0 ; i < query.clauses().size(); i++) {
|
||||
BooleanClause c = query.clauses().get(i);
|
||||
final boolean queryNeedsScores = needsScores && c.getOccur() != Occur.MUST_NOT;
|
||||
Weight w = c.getQuery().createWeight(searcher, queryNeedsScores);
|
||||
Weight w = c.getQuery().createWeight(searcher, needsScores && c.isScoring());
|
||||
weights.add(w);
|
||||
if (!c.isProhibited()) {
|
||||
if (c.isScoring()) {
|
||||
maxCoord++;
|
||||
}
|
||||
}
|
||||
|
@ -67,8 +66,8 @@ public class BooleanWeight extends Weight {
|
|||
for (int i = 0 ; i < weights.size(); i++) {
|
||||
// call sumOfSquaredWeights for all clauses in case of side effects
|
||||
float s = weights.get(i).getValueForNormalization(); // sum sub weights
|
||||
if (!query.clauses().get(i).isProhibited()) {
|
||||
// only add to sum for non-prohibited clauses
|
||||
if (query.clauses().get(i).isScoring()) {
|
||||
// only add to sum for scoring clauses
|
||||
sum += s;
|
||||
}
|
||||
}
|
||||
|
@ -79,18 +78,26 @@ public class BooleanWeight extends Weight {
|
|||
}
|
||||
|
||||
public float coord(int overlap, int maxOverlap) {
|
||||
// LUCENE-4300: in most cases of maxOverlap=1, BQ rewrites itself away,
|
||||
// so coord() is not applied. But when BQ cannot optimize itself away
|
||||
// for a single clause (minNrShouldMatch, prohibited clauses, etc), it's
|
||||
// important not to apply coord(1,1) for consistency, it might not be 1.0F
|
||||
return maxOverlap == 1 ? 1F : similarity.coord(overlap, maxOverlap);
|
||||
if (overlap == 0) {
|
||||
// special case that there are only non-scoring clauses
|
||||
return 0F;
|
||||
} else if (maxOverlap == 1) {
|
||||
// LUCENE-4300: in most cases of maxOverlap=1, BQ rewrites itself away,
|
||||
// so coord() is not applied. But when BQ cannot optimize itself away
|
||||
// for a single clause (minNrShouldMatch, prohibited clauses, etc), it's
|
||||
// important not to apply coord(1,1) for consistency, it might not be 1.0F
|
||||
return 1F;
|
||||
} else {
|
||||
// common case: use the similarity to compute the coord
|
||||
return similarity.coord(overlap, maxOverlap);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void normalize(float norm, float topLevelBoost) {
|
||||
topLevelBoost *= query.getBoost(); // incorporate boost
|
||||
for (Weight w : weights) {
|
||||
// normalize all clauses, (even if prohibited in case of side affects)
|
||||
// normalize all clauses, (even if non-scoring in case of side affects)
|
||||
w.normalize(norm, topLevelBoost);
|
||||
}
|
||||
}
|
||||
|
@ -118,11 +125,16 @@ public class BooleanWeight extends Weight {
|
|||
}
|
||||
Explanation e = w.explain(context, doc);
|
||||
if (e.isMatch()) {
|
||||
if (!c.isProhibited()) {
|
||||
if (c.isScoring()) {
|
||||
sumExpl.addDetail(e);
|
||||
sum += e.getValue();
|
||||
coord++;
|
||||
} else {
|
||||
} else if (c.isRequired()) {
|
||||
Explanation r =
|
||||
new Explanation(0.0f, "match on required clause (" + c.getQuery().toString() + ")");
|
||||
r.addDetail(e);
|
||||
sumExpl.addDetail(r);
|
||||
} else if (c.isProhibited()) {
|
||||
Explanation r =
|
||||
new Explanation(0.0f, "match on prohibited clause (" + c.getQuery().toString() + ")");
|
||||
r.addDetail(e);
|
||||
|
@ -243,6 +255,8 @@ public class BooleanWeight extends Weight {
|
|||
int minShouldMatch = query.minNrShouldMatch;
|
||||
|
||||
List<Scorer> required = new ArrayList<>();
|
||||
// clauses that are required AND participate in scoring, subset of 'required'
|
||||
List<Scorer> requiredScoring = new ArrayList<>();
|
||||
List<Scorer> prohibited = new ArrayList<>();
|
||||
List<Scorer> optional = new ArrayList<>();
|
||||
Iterator<BooleanClause> cIter = query.clauses().iterator();
|
||||
|
@ -255,6 +269,9 @@ public class BooleanWeight extends Weight {
|
|||
}
|
||||
} else if (c.isRequired()) {
|
||||
required.add(subScorer);
|
||||
if (c.isScoring()) {
|
||||
requiredScoring.add(subScorer);
|
||||
}
|
||||
} else if (c.isProhibited()) {
|
||||
prohibited.add(subScorer);
|
||||
} else {
|
||||
|
@ -267,6 +284,7 @@ public class BooleanWeight extends Weight {
|
|||
if (optional.size() == minShouldMatch) {
|
||||
// any optional clauses are in fact required
|
||||
required.addAll(optional);
|
||||
requiredScoring.addAll(optional);
|
||||
optional.clear();
|
||||
minShouldMatch = 0;
|
||||
}
|
||||
|
@ -280,7 +298,7 @@ public class BooleanWeight extends Weight {
|
|||
// no documents will be matched by the query
|
||||
return null;
|
||||
}
|
||||
|
||||
|
||||
// we don't need scores, so if we have required clauses, drop optional clauses completely
|
||||
if (!needsScores && minShouldMatch == 0 && required.size() > 0) {
|
||||
optional.clear();
|
||||
|
@ -290,7 +308,7 @@ public class BooleanWeight extends Weight {
|
|||
|
||||
// pure conjunction
|
||||
if (optional.isEmpty()) {
|
||||
return excl(req(required, disableCoord), prohibited);
|
||||
return excl(req(required, requiredScoring, disableCoord), prohibited);
|
||||
}
|
||||
|
||||
// pure disjunction
|
||||
|
@ -304,45 +322,49 @@ public class BooleanWeight extends Weight {
|
|||
// optional side must match. otherwise it's required + optional, factoring the
|
||||
// number of optional terms into the coord calculation
|
||||
|
||||
Scorer req = excl(req(required, true), prohibited);
|
||||
Scorer req = excl(req(required, requiredScoring, true), prohibited);
|
||||
Scorer opt = opt(optional, minShouldMatch, true);
|
||||
|
||||
// TODO: clean this up: it's horrible
|
||||
if (disableCoord) {
|
||||
if (minShouldMatch > 0) {
|
||||
return new ConjunctionScorer(this, new Scorer[] { req, opt }, 1F);
|
||||
return new ConjunctionScorer(this, Arrays.asList(req, opt), Arrays.asList(req, opt), 1F);
|
||||
} else {
|
||||
return new ReqOptSumScorer(req, opt);
|
||||
}
|
||||
} else if (optional.size() == 1) {
|
||||
if (minShouldMatch > 0) {
|
||||
return new ConjunctionScorer(this, new Scorer[] { req, opt }, coord(required.size()+1, maxCoord));
|
||||
return new ConjunctionScorer(this, Arrays.asList(req, opt), Arrays.asList(req, opt), coord(requiredScoring.size()+1, maxCoord));
|
||||
} else {
|
||||
float coordReq = coord(required.size(), maxCoord);
|
||||
float coordBoth = coord(required.size() + 1, maxCoord);
|
||||
float coordReq = coord(requiredScoring.size(), maxCoord);
|
||||
float coordBoth = coord(requiredScoring.size() + 1, maxCoord);
|
||||
return new BooleanTopLevelScorers.ReqSingleOptScorer(req, opt, coordReq, coordBoth);
|
||||
}
|
||||
} else {
|
||||
if (minShouldMatch > 0) {
|
||||
return new BooleanTopLevelScorers.CoordinatingConjunctionScorer(this, coords(), req, required.size(), opt);
|
||||
return new BooleanTopLevelScorers.CoordinatingConjunctionScorer(this, coords(), req, requiredScoring.size(), opt);
|
||||
} else {
|
||||
return new BooleanTopLevelScorers.ReqMultiOptScorer(req, opt, required.size(), coords());
|
||||
return new BooleanTopLevelScorers.ReqMultiOptScorer(req, opt, requiredScoring.size(), coords());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private Scorer req(List<Scorer> required, boolean disableCoord) {
|
||||
|
||||
/** Create a new scorer for the given required clauses. Note that
|
||||
* {@code requiredScoring} is a subset of {@code required} containing
|
||||
* required clauses that should participate in scoring. */
|
||||
private Scorer req(List<Scorer> required, List<Scorer> requiredScoring, boolean disableCoord) {
|
||||
if (required.size() == 1) {
|
||||
Scorer req = required.get(0);
|
||||
if (!disableCoord && maxCoord > 1) {
|
||||
return new BooleanTopLevelScorers.BoostedScorer(req, coord(1, maxCoord));
|
||||
} else {
|
||||
|
||||
if (needsScores == false ||
|
||||
(requiredScoring.size() == 1 && (disableCoord || maxCoord == 1))) {
|
||||
return req;
|
||||
} else {
|
||||
return new BooleanTopLevelScorers.BoostedScorer(req, coord(requiredScoring.size(), maxCoord));
|
||||
}
|
||||
} else {
|
||||
return new ConjunctionScorer(this,
|
||||
required.toArray(new Scorer[required.size()]),
|
||||
disableCoord ? 1.0F : coord(required.size(), maxCoord));
|
||||
return new ConjunctionScorer(this, required, requiredScoring,
|
||||
disableCoord ? 1.0F : coord(requiredScoring.size(), maxCoord));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -21,6 +21,7 @@ import java.io.IOException;
|
|||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
@ -31,18 +32,21 @@ class ConjunctionScorer extends Scorer {
|
|||
protected int lastDoc = -1;
|
||||
protected final DocsAndFreqs[] docsAndFreqs;
|
||||
private final DocsAndFreqs lead;
|
||||
private final Scorer[] scorers;
|
||||
private final float coord;
|
||||
|
||||
ConjunctionScorer(Weight weight, Scorer[] scorers) {
|
||||
this(weight, scorers, 1f);
|
||||
ConjunctionScorer(Weight weight, List<? extends DocIdSetIterator> required, List<Scorer> scorers) {
|
||||
this(weight, required, scorers, 1f);
|
||||
}
|
||||
|
||||
ConjunctionScorer(Weight weight, Scorer[] scorers, float coord) {
|
||||
/** Create a new {@link ConjunctionScorer}, note that {@code scorers} must be a subset of {@code required}. */
|
||||
ConjunctionScorer(Weight weight, List<? extends DocIdSetIterator> required, List<Scorer> scorers, float coord) {
|
||||
super(weight);
|
||||
assert required.containsAll(scorers);
|
||||
this.coord = coord;
|
||||
this.docsAndFreqs = new DocsAndFreqs[scorers.length];
|
||||
for (int i = 0; i < scorers.length; i++) {
|
||||
docsAndFreqs[i] = new DocsAndFreqs(scorers[i]);
|
||||
this.docsAndFreqs = new DocsAndFreqs[required.size()];
|
||||
for (int i = 0; i < required.size(); ++i) {
|
||||
docsAndFreqs[i] = new DocsAndFreqs(required.get(i));
|
||||
}
|
||||
// Sort the array the first time to allow the least frequent DocsEnum to
|
||||
// lead the matching.
|
||||
|
@ -54,6 +58,8 @@ class ConjunctionScorer extends Scorer {
|
|||
});
|
||||
|
||||
lead = docsAndFreqs[0]; // least frequent DocsEnum leads the intersection
|
||||
|
||||
this.scorers = scorers.toArray(new Scorer[scorers.size()]);
|
||||
}
|
||||
|
||||
private int doNext(int doc) throws IOException {
|
||||
|
@ -68,7 +74,7 @@ class ConjunctionScorer extends Scorer {
|
|||
// docsAndFreqs[i].doc may already be equal to doc if we "broke advanceHead"
|
||||
// on the previous iteration and the advance on the lead scorer exactly matched.
|
||||
if (docsAndFreqs[i].doc < doc) {
|
||||
docsAndFreqs[i].doc = docsAndFreqs[i].scorer.advance(doc);
|
||||
docsAndFreqs[i].doc = docsAndFreqs[i].iterator.advance(doc);
|
||||
|
||||
if (docsAndFreqs[i].doc > doc) {
|
||||
// DocsEnum beyond the current doc - break and advance lead to the new highest doc.
|
||||
|
@ -81,13 +87,13 @@ class ConjunctionScorer extends Scorer {
|
|||
return doc;
|
||||
}
|
||||
// advance head for next iteration
|
||||
doc = lead.doc = lead.scorer.advance(doc);
|
||||
doc = lead.doc = lead.iterator.advance(doc);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int advance(int target) throws IOException {
|
||||
lead.doc = lead.scorer.advance(target);
|
||||
lead.doc = lead.iterator.advance(target);
|
||||
return lastDoc = doNext(lead.doc);
|
||||
}
|
||||
|
||||
|
@ -98,7 +104,7 @@ class ConjunctionScorer extends Scorer {
|
|||
|
||||
@Override
|
||||
public int nextDoc() throws IOException {
|
||||
lead.doc = lead.scorer.nextDoc();
|
||||
lead.doc = lead.iterator.nextDoc();
|
||||
return lastDoc = doNext(lead.doc);
|
||||
}
|
||||
|
||||
|
@ -106,8 +112,8 @@ class ConjunctionScorer extends Scorer {
|
|||
public float score() throws IOException {
|
||||
// TODO: sum into a double and cast to float if we ever send required clauses to BS1
|
||||
float sum = 0.0f;
|
||||
for (DocsAndFreqs docs : docsAndFreqs) {
|
||||
sum += docs.scorer.score();
|
||||
for (Scorer scorer : scorers) {
|
||||
sum += scorer.score();
|
||||
}
|
||||
return sum * coord;
|
||||
}
|
||||
|
@ -139,26 +145,26 @@ class ConjunctionScorer extends Scorer {
|
|||
|
||||
@Override
|
||||
public long cost() {
|
||||
return lead.scorer.cost();
|
||||
return lead.iterator.cost();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<ChildScorer> getChildren() {
|
||||
ArrayList<ChildScorer> children = new ArrayList<>(docsAndFreqs.length);
|
||||
for (DocsAndFreqs docs : docsAndFreqs) {
|
||||
children.add(new ChildScorer(docs.scorer, "MUST"));
|
||||
for (Scorer scorer : scorers) {
|
||||
children.add(new ChildScorer(scorer, "MUST"));
|
||||
}
|
||||
return children;
|
||||
}
|
||||
|
||||
static final class DocsAndFreqs {
|
||||
final long cost;
|
||||
final Scorer scorer;
|
||||
final DocIdSetIterator iterator;
|
||||
int doc = -1;
|
||||
|
||||
DocsAndFreqs(Scorer scorer) {
|
||||
this.scorer = scorer;
|
||||
this.cost = scorer.cost();
|
||||
DocsAndFreqs(DocIdSetIterator iterator) {
|
||||
this.iterator = iterator;
|
||||
this.cost = iterator.cost();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,12 +17,15 @@ package org.apache.lucene.search;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.BitSet;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
|
@ -33,9 +36,11 @@ 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.LeafReaderContext;
|
||||
import org.apache.lucene.index.MultiReader;
|
||||
import org.apache.lucene.index.RandomIndexWriter;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.search.BooleanClause.Occur;
|
||||
import org.apache.lucene.search.similarities.DefaultSimilarity;
|
||||
import org.apache.lucene.search.spans.SpanQuery;
|
||||
import org.apache.lucene.search.spans.SpanTermQuery;
|
||||
|
@ -45,7 +50,7 @@ import org.apache.lucene.util.NamedThreadFactory;
|
|||
import org.apache.lucene.util.TestUtil;
|
||||
|
||||
public class TestBooleanQuery extends LuceneTestCase {
|
||||
|
||||
|
||||
public void testEquality() throws Exception {
|
||||
BooleanQuery bq1 = new BooleanQuery();
|
||||
bq1.add(new TermQuery(new Term("field", "value1")), BooleanClause.Occur.SHOULD);
|
||||
|
@ -116,7 +121,7 @@ public class TestBooleanQuery extends LuceneTestCase {
|
|||
q.add(subQuery, BooleanClause.Occur.MUST);
|
||||
score2 = s.search(q, 10).getMaxScore();
|
||||
assertEquals(score*(2/3F), score2, 1e-6);
|
||||
|
||||
|
||||
// PhraseQuery w/ no terms added returns a null scorer
|
||||
PhraseQuery pq = new PhraseQuery();
|
||||
q.add(pq, BooleanClause.Occur.SHOULD);
|
||||
|
@ -134,7 +139,7 @@ public class TestBooleanQuery extends LuceneTestCase {
|
|||
dmq.add(new TermQuery(new Term("field", "a")));
|
||||
dmq.add(pq);
|
||||
assertEquals(1, s.search(dmq, 10).totalHits);
|
||||
|
||||
|
||||
r.close();
|
||||
w.close();
|
||||
dir.close();
|
||||
|
@ -148,7 +153,7 @@ public class TestBooleanQuery extends LuceneTestCase {
|
|||
iw1.addDocument(doc1);
|
||||
IndexReader reader1 = iw1.getReader();
|
||||
iw1.close();
|
||||
|
||||
|
||||
Directory dir2 = newDirectory();
|
||||
RandomIndexWriter iw2 = new RandomIndexWriter(random(), dir2);
|
||||
Document doc2 = new Document();
|
||||
|
@ -162,11 +167,11 @@ public class TestBooleanQuery extends LuceneTestCase {
|
|||
WildcardQuery wildcardQuery = new WildcardQuery(new Term("field", "ba*"));
|
||||
wildcardQuery.setRewriteMethod(MultiTermQuery.SCORING_BOOLEAN_QUERY_REWRITE);
|
||||
query.add(wildcardQuery, BooleanClause.Occur.MUST_NOT);
|
||||
|
||||
|
||||
MultiReader multireader = new MultiReader(reader1, reader2);
|
||||
IndexSearcher searcher = newSearcher(multireader);
|
||||
assertEquals(0, searcher.search(query, 10).totalHits);
|
||||
|
||||
|
||||
final ExecutorService es = Executors.newCachedThreadPool(new NamedThreadFactory("NRT search threads"));
|
||||
searcher = new IndexSearcher(multireader, es);
|
||||
if (VERBOSE)
|
||||
|
@ -284,7 +289,7 @@ public class TestBooleanQuery extends LuceneTestCase {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
r.close();
|
||||
d.close();
|
||||
}
|
||||
|
@ -328,7 +333,7 @@ public class TestBooleanQuery extends LuceneTestCase {
|
|||
final float BOOST = 3.5F;
|
||||
final String FIELD = "content";
|
||||
final String VALUE = "foo";
|
||||
|
||||
|
||||
Directory dir = newDirectory();
|
||||
(new RandomIndexWriter(random(), dir)).close();
|
||||
IndexReader r = DirectoryReader.open(dir);
|
||||
|
@ -347,7 +352,7 @@ public class TestBooleanQuery extends LuceneTestCase {
|
|||
}
|
||||
|
||||
BooleanQuery bq = new BooleanQuery();
|
||||
bq.add(actual, random().nextBoolean()
|
||||
bq.add(actual, random().nextBoolean()
|
||||
? BooleanClause.Occur.SHOULD : BooleanClause.Occur.MUST);
|
||||
actual = bq;
|
||||
}
|
||||
|
@ -381,4 +386,208 @@ public class TestBooleanQuery extends LuceneTestCase {
|
|||
w.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
private static BitSet getMatches(IndexSearcher searcher, Query query) throws IOException {
|
||||
BitSet set = new BitSet();
|
||||
searcher.search(query, new SimpleCollector() {
|
||||
int docBase = 0;
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return random().nextBoolean();
|
||||
}
|
||||
@Override
|
||||
protected void doSetNextReader(LeafReaderContext context)
|
||||
throws IOException {
|
||||
super.doSetNextReader(context);
|
||||
docBase = context.docBase;
|
||||
}
|
||||
@Override
|
||||
public void collect(int doc) throws IOException {
|
||||
set.set(docBase + doc);
|
||||
}
|
||||
});
|
||||
return set;
|
||||
}
|
||||
|
||||
public void testFILTERClauseBehavesLikeMUST() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
RandomIndexWriter w = new RandomIndexWriter(random(), dir);
|
||||
Document doc = new Document();
|
||||
Field f = newTextField("field", "a b c d", Field.Store.NO);
|
||||
doc.add(f);
|
||||
w.addDocument(doc);
|
||||
f.setStringValue("b d");
|
||||
w.addDocument(doc);
|
||||
f.setStringValue("d");
|
||||
w.addDocument(doc);
|
||||
w.commit();
|
||||
|
||||
DirectoryReader reader = w.getReader();
|
||||
final IndexSearcher searcher = new IndexSearcher(reader);
|
||||
|
||||
for (List<String> requiredTerms : Arrays.<List<String>>asList(
|
||||
Arrays.asList("a", "d"),
|
||||
Arrays.asList("a", "b", "d"),
|
||||
Arrays.asList("d"),
|
||||
Arrays.asList("e"),
|
||||
Arrays.asList())) {
|
||||
final BooleanQuery bq1 = new BooleanQuery();
|
||||
final BooleanQuery bq2 = new BooleanQuery();
|
||||
for (String term : requiredTerms) {
|
||||
final Query q = new TermQuery(new Term("field", term));
|
||||
bq1.add(q, Occur.MUST);
|
||||
bq2.add(q, Occur.FILTER);
|
||||
}
|
||||
|
||||
final BitSet matches1 = getMatches(searcher, bq1);
|
||||
final BitSet matches2 = getMatches(searcher, bq2);
|
||||
assertEquals(matches1, matches2);
|
||||
}
|
||||
|
||||
reader.close();
|
||||
w.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
private void assertSameScoresWithoutFilters(IndexSearcher searcher, BooleanQuery bq) throws IOException {
|
||||
final BooleanQuery bq2 = new BooleanQuery();
|
||||
for (BooleanClause c : bq.getClauses()) {
|
||||
if (c.getOccur() != Occur.FILTER) {
|
||||
bq2.add(c);
|
||||
}
|
||||
}
|
||||
bq2.setMinimumNumberShouldMatch(bq.getMinimumNumberShouldMatch());
|
||||
bq2.setBoost(bq.getBoost());
|
||||
|
||||
final AtomicBoolean matched = new AtomicBoolean();
|
||||
searcher.search(bq, new SimpleCollector() {
|
||||
int docBase;
|
||||
Scorer scorer;
|
||||
|
||||
@Override
|
||||
protected void doSetNextReader(LeafReaderContext context)
|
||||
throws IOException {
|
||||
super.doSetNextReader(context);
|
||||
docBase = context.docBase;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setScorer(Scorer scorer) throws IOException {
|
||||
this.scorer = scorer;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void collect(int doc) throws IOException {
|
||||
final float actualScore = scorer.score();
|
||||
final float expectedScore = searcher.explain(bq2, docBase + doc).getValue();
|
||||
assertEquals(expectedScore, actualScore, 10e-5);
|
||||
matched.set(true);
|
||||
}
|
||||
});
|
||||
assertTrue(matched.get());
|
||||
}
|
||||
|
||||
public void testFilterClauseDoesNotImpactScore() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
RandomIndexWriter w = new RandomIndexWriter(random(), dir);
|
||||
Document doc = new Document();
|
||||
Field f = newTextField("field", "a b c d", Field.Store.NO);
|
||||
doc.add(f);
|
||||
w.addDocument(doc);
|
||||
f.setStringValue("b d");
|
||||
w.addDocument(doc);
|
||||
f.setStringValue("a d");
|
||||
w.addDocument(doc);
|
||||
w.commit();
|
||||
|
||||
DirectoryReader reader = w.getReader();
|
||||
final IndexSearcher searcher = new IndexSearcher(reader);
|
||||
|
||||
BooleanQuery q = new BooleanQuery();
|
||||
q.setBoost(random().nextFloat());
|
||||
q.add(new TermQuery(new Term("field", "a")), Occur.FILTER);
|
||||
|
||||
// With a single clause, we will rewrite to the underlying
|
||||
// query. Make sure that it returns null scores
|
||||
assertSameScoresWithoutFilters(searcher, q);
|
||||
|
||||
// Now with two clauses, we will get a conjunction scorer
|
||||
// Make sure it returns null scores
|
||||
q.add(new TermQuery(new Term("field", "b")), Occur.FILTER);
|
||||
assertSameScoresWithoutFilters(searcher, q);
|
||||
|
||||
// Now with a scoring clause, we need to make sure that
|
||||
// the boolean scores are the same as those from the term
|
||||
// query
|
||||
q.add(new TermQuery(new Term("field", "c")), Occur.SHOULD);
|
||||
assertSameScoresWithoutFilters(searcher, q);
|
||||
|
||||
// FILTER and empty SHOULD
|
||||
q = new BooleanQuery();
|
||||
q.setBoost(random().nextFloat());
|
||||
q.add(new TermQuery(new Term("field", "a")), Occur.FILTER);
|
||||
q.add(new TermQuery(new Term("field", "e")), Occur.SHOULD);
|
||||
assertSameScoresWithoutFilters(searcher, q);
|
||||
|
||||
// mix of FILTER and MUST
|
||||
q = new BooleanQuery();
|
||||
q.setBoost(random().nextFloat());
|
||||
q.add(new TermQuery(new Term("field", "a")), Occur.FILTER);
|
||||
q.add(new TermQuery(new Term("field", "d")), Occur.MUST);
|
||||
assertSameScoresWithoutFilters(searcher, q);
|
||||
|
||||
// FILTER + minShouldMatch
|
||||
q = new BooleanQuery();
|
||||
q.setBoost(random().nextFloat());
|
||||
q.add(new TermQuery(new Term("field", "b")), Occur.FILTER);
|
||||
q.add(new TermQuery(new Term("field", "a")), Occur.SHOULD);
|
||||
q.add(new TermQuery(new Term("field", "d")), Occur.SHOULD);
|
||||
q.setMinimumNumberShouldMatch(1);
|
||||
assertSameScoresWithoutFilters(searcher, q);
|
||||
|
||||
reader.close();
|
||||
w.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public void testSingleFilterClause() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
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();
|
||||
final IndexSearcher searcher = new IndexSearcher(reader);
|
||||
|
||||
BooleanQuery query1 = new BooleanQuery();
|
||||
query1.add(new TermQuery(new Term("field", "a")), Occur.FILTER);
|
||||
|
||||
// Single clauses rewrite to a term query
|
||||
final Query rewritten1 = query1.rewrite(reader);
|
||||
assertTrue(rewritten1 instanceof TermQuery);
|
||||
assertEquals(0f, rewritten1.getBoost(), 0f);
|
||||
|
||||
// When there are two clauses, we cannot rewrite, but if one of them creates
|
||||
// a null scorer we will end up with a single filter scorer and will need to
|
||||
// make sure to set score=0
|
||||
BooleanQuery query2 = new BooleanQuery();
|
||||
query2.add(new TermQuery(new Term("field", "a")), Occur.FILTER);
|
||||
query2.add(new TermQuery(new Term("field", "b")), Occur.SHOULD);
|
||||
final Weight weight = searcher.createNormalizedWeight(query2, true);
|
||||
final Scorer scorer = weight.scorer(reader.leaves().get(0), null);
|
||||
assertTrue(scorer.getClass().getName(), scorer instanceof BooleanTopLevelScorers.BoostedScorer);
|
||||
assertEquals(0, ((BooleanTopLevelScorers.BoostedScorer) scorer).boost, 0f);
|
||||
|
||||
reader.close();
|
||||
w.close();
|
||||
dir.close();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -198,11 +198,12 @@ public class TestBooleanQueryVisitSubscorers extends LuceneTestCase {
|
|||
for (String summary : collector.getSummaries()) {
|
||||
assertEquals(
|
||||
"CoordinatingConjunctionScorer\n" +
|
||||
" MUST MatchAllScorer\n" +
|
||||
" MUST MinShouldMatchSumScorer\n" +
|
||||
" SHOULD TermScorer body:nutch\n" +
|
||||
" SHOULD TermScorer body:web\n" +
|
||||
" SHOULD TermScorer body:crawler\n" +
|
||||
" MUST MatchAllScorer", summary);
|
||||
" SHOULD TermScorer body:crawler",
|
||||
summary);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue