mirror of https://github.com/apache/lucene.git
LUCENE-6226: Revert for more API discussions
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1658375 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
ef2141f6bf
commit
e0ff51383c
|
@ -88,10 +88,6 @@ API Changes
|
||||||
* LUCENE-6223: Move BooleanQuery.BooleanWeight to BooleanWeight.
|
* LUCENE-6223: Move BooleanQuery.BooleanWeight to BooleanWeight.
|
||||||
(Robert Muir)
|
(Robert Muir)
|
||||||
|
|
||||||
* LUCENE-6226: Query.createWeight() and IndexSearcher.createNormalizedWeight()
|
|
||||||
now take an integer bitmask indicating which postings values should be
|
|
||||||
retrieved from the index. (Alan Woodward, Adrien Grand)
|
|
||||||
|
|
||||||
Other
|
Other
|
||||||
|
|
||||||
* LUCENE-6193: Collapse identical catch branches in try-catch statements.
|
* LUCENE-6193: Collapse identical catch branches in try-catch statements.
|
||||||
|
|
|
@ -92,16 +92,17 @@ public abstract class PostingsEnum extends DocIdSetIterator {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns the next position, or -1 if positions are not available.
|
* Returns the next position. If there are no more
|
||||||
* Should only be called up to freq() times */
|
* positions, or the iterator does not support positions,
|
||||||
|
* this will return DocsEnum.NO_MORE_POSITIONS */
|
||||||
public abstract int nextPosition() throws IOException;
|
public abstract int nextPosition() throws IOException;
|
||||||
|
|
||||||
/** Returns start offset for the current position, or -1
|
/** Returns start offset for the current position, or -1
|
||||||
* if offsets are not available. */
|
* if offsets were not indexed. */
|
||||||
public abstract int startOffset() throws IOException;
|
public abstract int startOffset() throws IOException;
|
||||||
|
|
||||||
/** Returns end offset for the current position, or -1 if
|
/** Returns end offset for the current position, or -1 if
|
||||||
* offsets are not available. */
|
* offsets were not indexed. */
|
||||||
public abstract int endOffset() throws IOException;
|
public abstract int endOffset() throws IOException;
|
||||||
|
|
||||||
/** Returns the payload at this position, or null if no
|
/** Returns the payload at this position, or null if no
|
||||||
|
|
|
@ -163,8 +163,8 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
|
||||||
public final Iterator<BooleanClause> iterator() { return clauses().iterator(); }
|
public final Iterator<BooleanClause> iterator() { return clauses().iterator(); }
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Weight createWeight(IndexSearcher searcher, int postingsFlags) throws IOException {
|
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
|
||||||
return new BooleanWeight(this, searcher, postingsFlags, disableCoord);
|
return new BooleanWeight(this, searcher, needsScores, disableCoord);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -24,7 +24,6 @@ import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
import org.apache.lucene.index.LeafReaderContext;
|
import org.apache.lucene.index.LeafReaderContext;
|
||||||
import org.apache.lucene.index.PostingsEnum;
|
|
||||||
import org.apache.lucene.search.BooleanClause.Occur;
|
import org.apache.lucene.search.BooleanClause.Occur;
|
||||||
import org.apache.lucene.search.similarities.Similarity;
|
import org.apache.lucene.search.similarities.Similarity;
|
||||||
import org.apache.lucene.util.Bits;
|
import org.apache.lucene.util.Bits;
|
||||||
|
@ -44,17 +43,17 @@ public class BooleanWeight extends Weight {
|
||||||
private final boolean disableCoord;
|
private final boolean disableCoord;
|
||||||
private final boolean needsScores;
|
private final boolean needsScores;
|
||||||
|
|
||||||
public BooleanWeight(BooleanQuery query, IndexSearcher searcher, int postingsFlags, boolean disableCoord) throws IOException {
|
public BooleanWeight(BooleanQuery query, IndexSearcher searcher, boolean needsScores, boolean disableCoord) throws IOException {
|
||||||
super(query);
|
super(query);
|
||||||
this.query = query;
|
this.query = query;
|
||||||
this.needsScores = (postingsFlags & PostingsEnum.FLAG_FREQS) != 0;
|
this.needsScores = needsScores;
|
||||||
this.similarity = searcher.getSimilarity();
|
this.similarity = searcher.getSimilarity();
|
||||||
this.disableCoord = disableCoord;
|
this.disableCoord = disableCoord;
|
||||||
weights = new ArrayList<>(query.clauses().size());
|
weights = new ArrayList<>(query.clauses().size());
|
||||||
for (int i = 0 ; i < query.clauses().size(); i++) {
|
for (int i = 0 ; i < query.clauses().size(); i++) {
|
||||||
BooleanClause c = query.clauses().get(i);
|
BooleanClause c = query.clauses().get(i);
|
||||||
final int subQueryFlags = c.getOccur() == Occur.MUST_NOT ? PostingsEnum.FLAG_NONE : postingsFlags;
|
final boolean queryNeedsScores = needsScores && c.getOccur() != Occur.MUST_NOT;
|
||||||
Weight w = c.getQuery().createWeight(searcher, subQueryFlags);
|
Weight w = c.getQuery().createWeight(searcher, queryNeedsScores);
|
||||||
weights.add(w);
|
weights.add(w);
|
||||||
if (!c.isProhibited()) {
|
if (!c.isProhibited()) {
|
||||||
maxCoord++;
|
maxCoord++;
|
||||||
|
|
|
@ -24,7 +24,6 @@ import java.util.Set;
|
||||||
|
|
||||||
import org.apache.lucene.index.IndexReader;
|
import org.apache.lucene.index.IndexReader;
|
||||||
import org.apache.lucene.index.LeafReaderContext;
|
import org.apache.lucene.index.LeafReaderContext;
|
||||||
import org.apache.lucene.index.PostingsEnum;
|
|
||||||
import org.apache.lucene.index.Term;
|
import org.apache.lucene.index.Term;
|
||||||
import org.apache.lucene.util.Bits;
|
import org.apache.lucene.util.Bits;
|
||||||
import org.apache.lucene.util.BytesRef;
|
import org.apache.lucene.util.BytesRef;
|
||||||
|
@ -112,7 +111,7 @@ public class ConstantScoreQuery extends Query {
|
||||||
|
|
||||||
public ConstantWeight(IndexSearcher searcher) throws IOException {
|
public ConstantWeight(IndexSearcher searcher) throws IOException {
|
||||||
super(ConstantScoreQuery.this);
|
super(ConstantScoreQuery.this);
|
||||||
this.innerWeight = (query == null) ? null : query.createWeight(searcher, PostingsEnum.FLAG_NONE);
|
this.innerWeight = (query == null) ? null : query.createWeight(searcher, false);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -328,7 +327,7 @@ public class ConstantScoreQuery extends Query {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Weight createWeight(IndexSearcher searcher, int postingsFlags) throws IOException {
|
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
|
||||||
return new ConstantScoreQuery.ConstantWeight(searcher);
|
return new ConstantScoreQuery.ConstantWeight(searcher);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -118,10 +118,10 @@ public class DisjunctionMaxQuery extends Query implements Iterable<Query> {
|
||||||
protected ArrayList<Weight> weights = new ArrayList<>(); // The Weight's for our subqueries, in 1-1 correspondence with disjuncts
|
protected ArrayList<Weight> weights = new ArrayList<>(); // The Weight's for our subqueries, in 1-1 correspondence with disjuncts
|
||||||
|
|
||||||
/** Construct the Weight for this Query searched by searcher. Recursively construct subquery weights. */
|
/** Construct the Weight for this Query searched by searcher. Recursively construct subquery weights. */
|
||||||
public DisjunctionMaxWeight(IndexSearcher searcher, int postingsFlags) throws IOException {
|
public DisjunctionMaxWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
|
||||||
super(DisjunctionMaxQuery.this);
|
super(DisjunctionMaxQuery.this);
|
||||||
for (Query disjunctQuery : disjuncts) {
|
for (Query disjunctQuery : disjuncts) {
|
||||||
weights.add(disjunctQuery.createWeight(searcher, postingsFlags));
|
weights.add(disjunctQuery.createWeight(searcher, needsScores));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -194,8 +194,8 @@ public class DisjunctionMaxQuery extends Query implements Iterable<Query> {
|
||||||
|
|
||||||
/** Create the Weight used to score us */
|
/** Create the Weight used to score us */
|
||||||
@Override
|
@Override
|
||||||
public Weight createWeight(IndexSearcher searcher, int postingsFlags) throws IOException {
|
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
|
||||||
return new DisjunctionMaxWeight(searcher, postingsFlags);
|
return new DisjunctionMaxWeight(searcher, needsScores);
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Optimize our representation and our subqueries representations
|
/** Optimize our representation and our subqueries representations
|
||||||
|
|
|
@ -78,8 +78,8 @@ public class FilteredQuery extends Query {
|
||||||
* This is accomplished by overriding the Scorer returned by the Weight.
|
* This is accomplished by overriding the Scorer returned by the Weight.
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public Weight createWeight(final IndexSearcher searcher, int postingsFlags) throws IOException {
|
public Weight createWeight(final IndexSearcher searcher, boolean needsScores) throws IOException {
|
||||||
final Weight weight = query.createWeight (searcher, postingsFlags);
|
final Weight weight = query.createWeight (searcher, needsScores);
|
||||||
return new Weight(FilteredQuery.this) {
|
return new Weight(FilteredQuery.this) {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -33,7 +33,6 @@ import org.apache.lucene.index.IndexReaderContext;
|
||||||
import org.apache.lucene.index.IndexWriter; // javadocs
|
import org.apache.lucene.index.IndexWriter; // javadocs
|
||||||
import org.apache.lucene.index.LeafReaderContext;
|
import org.apache.lucene.index.LeafReaderContext;
|
||||||
import org.apache.lucene.index.MultiFields;
|
import org.apache.lucene.index.MultiFields;
|
||||||
import org.apache.lucene.index.PostingsEnum;
|
|
||||||
import org.apache.lucene.index.ReaderUtil;
|
import org.apache.lucene.index.ReaderUtil;
|
||||||
import org.apache.lucene.index.StoredDocument;
|
import org.apache.lucene.index.StoredDocument;
|
||||||
import org.apache.lucene.index.StoredFieldVisitor;
|
import org.apache.lucene.index.StoredFieldVisitor;
|
||||||
|
@ -241,14 +240,13 @@ public class IndexSearcher {
|
||||||
return collector.topDocs();
|
return collector.topDocs();
|
||||||
} else {
|
} else {
|
||||||
final TopScoreDocCollector[] collectors = new TopScoreDocCollector[leafSlices.length];
|
final TopScoreDocCollector[] collectors = new TopScoreDocCollector[leafSlices.length];
|
||||||
int postingsFlags = PostingsEnum.FLAG_NONE;
|
boolean needsScores = false;
|
||||||
for (int i = 0; i < leafSlices.length; ++i) {
|
for (int i = 0; i < leafSlices.length; ++i) {
|
||||||
collectors[i] = TopScoreDocCollector.create(numHits, after);
|
collectors[i] = TopScoreDocCollector.create(numHits, after);
|
||||||
if (collectors[i].needsScores())
|
needsScores |= collectors[i].needsScores();
|
||||||
postingsFlags |= PostingsEnum.FLAG_FREQS;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
final Weight weight = createNormalizedWeight(query, postingsFlags);
|
final Weight weight = createNormalizedWeight(query, needsScores);
|
||||||
final List<Future<TopDocs>> topDocsFutures = new ArrayList<>(leafSlices.length);
|
final List<Future<TopDocs>> topDocsFutures = new ArrayList<>(leafSlices.length);
|
||||||
for (int i = 0; i < leafSlices.length; ++i) {
|
for (int i = 0; i < leafSlices.length; ++i) {
|
||||||
final LeafReaderContext[] leaves = leafSlices[i].leaves;
|
final LeafReaderContext[] leaves = leafSlices[i].leaves;
|
||||||
|
@ -340,8 +338,7 @@ public class IndexSearcher {
|
||||||
*/
|
*/
|
||||||
public void search(Query query, Collector results)
|
public void search(Query query, Collector results)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
int postingsFlags = results.needsScores() ? PostingsEnum.FLAG_FREQS : PostingsEnum.FLAG_NONE;
|
search(leafContexts, createNormalizedWeight(query, results.needsScores()), results);
|
||||||
search(leafContexts, createNormalizedWeight(query, postingsFlags), results);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Search implementation with arbitrary sorting. Finds
|
/** Search implementation with arbitrary sorting. Finds
|
||||||
|
@ -465,14 +462,13 @@ public class IndexSearcher {
|
||||||
return collector.topDocs();
|
return collector.topDocs();
|
||||||
} else {
|
} else {
|
||||||
final TopFieldCollector[] collectors = new TopFieldCollector[leafSlices.length];
|
final TopFieldCollector[] collectors = new TopFieldCollector[leafSlices.length];
|
||||||
int postingsFlags = PostingsEnum.FLAG_NONE;
|
boolean needsScores = false;
|
||||||
for (int i = 0; i < leafSlices.length; ++i) {
|
for (int i = 0; i < leafSlices.length; ++i) {
|
||||||
collectors[i] = TopFieldCollector.create(sort, numHits, after, fillFields, doDocScores, doMaxScore);
|
collectors[i] = TopFieldCollector.create(sort, numHits, after, fillFields, doDocScores, doMaxScore);
|
||||||
if (collectors[i].needsScores())
|
needsScores |= collectors[i].needsScores();
|
||||||
postingsFlags |= PostingsEnum.FLAG_FREQS;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
final Weight weight = createNormalizedWeight(query, postingsFlags);
|
final Weight weight = createNormalizedWeight(query, needsScores);
|
||||||
final List<Future<TopFieldDocs>> topDocsFutures = new ArrayList<>(leafSlices.length);
|
final List<Future<TopFieldDocs>> topDocsFutures = new ArrayList<>(leafSlices.length);
|
||||||
for (int i = 0; i < leafSlices.length; ++i) {
|
for (int i = 0; i < leafSlices.length; ++i) {
|
||||||
final LeafReaderContext[] leaves = leafSlices[i].leaves;
|
final LeafReaderContext[] leaves = leafSlices[i].leaves;
|
||||||
|
@ -569,7 +565,7 @@ public class IndexSearcher {
|
||||||
* entire index.
|
* entire index.
|
||||||
*/
|
*/
|
||||||
public Explanation explain(Query query, int doc) throws IOException {
|
public Explanation explain(Query query, int doc) throws IOException {
|
||||||
return explain(createNormalizedWeight(query, PostingsEnum.FLAG_FREQS), doc);
|
return explain(createNormalizedWeight(query, true), doc);
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Expert: low-level implementation method
|
/** Expert: low-level implementation method
|
||||||
|
@ -599,9 +595,9 @@ public class IndexSearcher {
|
||||||
* can then directly be used to get a {@link Scorer}.
|
* can then directly be used to get a {@link Scorer}.
|
||||||
* @lucene.internal
|
* @lucene.internal
|
||||||
*/
|
*/
|
||||||
public Weight createNormalizedWeight(Query query, int postingsFlags) throws IOException {
|
public Weight createNormalizedWeight(Query query, boolean needsScores) throws IOException {
|
||||||
query = rewrite(query);
|
query = rewrite(query);
|
||||||
Weight weight = query.createWeight(this, postingsFlags);
|
Weight weight = query.createWeight(this, needsScores);
|
||||||
float v = weight.getValueForNormalization();
|
float v = weight.getValueForNormalization();
|
||||||
float norm = getSimilarity().queryNorm(v);
|
float norm = getSimilarity().queryNorm(v);
|
||||||
if (Float.isInfinite(norm) || Float.isNaN(norm)) {
|
if (Float.isInfinite(norm) || Float.isNaN(norm)) {
|
||||||
|
|
|
@ -150,7 +150,7 @@ public class MatchAllDocsQuery extends Query {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Weight createWeight(IndexSearcher searcher, int postingsFlags) {
|
public Weight createWeight(IndexSearcher searcher, boolean needsScores) {
|
||||||
return new MatchAllDocsWeight(searcher);
|
return new MatchAllDocsWeight(searcher);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -142,13 +142,11 @@ public class MultiPhraseQuery extends Query {
|
||||||
private final Similarity.SimWeight stats;
|
private final Similarity.SimWeight stats;
|
||||||
private final Map<Term,TermContext> termContexts = new HashMap<>();
|
private final Map<Term,TermContext> termContexts = new HashMap<>();
|
||||||
private final boolean needsScores;
|
private final boolean needsScores;
|
||||||
private final int postingsFlags;
|
|
||||||
|
|
||||||
public MultiPhraseWeight(IndexSearcher searcher, int postingsFlags)
|
public MultiPhraseWeight(IndexSearcher searcher, boolean needsScores)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
super(MultiPhraseQuery.this);
|
super(MultiPhraseQuery.this);
|
||||||
this.needsScores = (postingsFlags & PostingsEnum.FLAG_FREQS) != 0;
|
this.needsScores = needsScores;
|
||||||
this.postingsFlags = postingsFlags | PostingsEnum.FLAG_POSITIONS;
|
|
||||||
this.similarity = searcher.getSimilarity();
|
this.similarity = searcher.getSimilarity();
|
||||||
final IndexReaderContext context = searcher.getTopReaderContext();
|
final IndexReaderContext context = searcher.getTopReaderContext();
|
||||||
|
|
||||||
|
@ -230,7 +228,7 @@ public class MultiPhraseQuery extends Query {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
termsEnum.seekExact(term.bytes(), termState);
|
termsEnum.seekExact(term.bytes(), termState);
|
||||||
postingsEnum = termsEnum.postings(liveDocs, null, postingsFlags);
|
postingsEnum = termsEnum.postings(liveDocs, null, PostingsEnum.FLAG_POSITIONS);
|
||||||
|
|
||||||
if (postingsEnum == null) {
|
if (postingsEnum == null) {
|
||||||
// term does exist, but has no positions
|
// term does exist, but has no positions
|
||||||
|
@ -298,8 +296,8 @@ public class MultiPhraseQuery extends Query {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Weight createWeight(IndexSearcher searcher, int postingsFlags) throws IOException {
|
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
|
||||||
return new MultiPhraseWeight(searcher, postingsFlags);
|
return new MultiPhraseWeight(searcher, needsScores);
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Prints a user-readable version of this query. */
|
/** Prints a user-readable version of this query. */
|
||||||
|
|
|
@ -211,15 +211,13 @@ public class PhraseQuery extends Query {
|
||||||
private class PhraseWeight extends Weight {
|
private class PhraseWeight extends Weight {
|
||||||
private final Similarity similarity;
|
private final Similarity similarity;
|
||||||
private final Similarity.SimWeight stats;
|
private final Similarity.SimWeight stats;
|
||||||
private final int postingsFlags;
|
|
||||||
private final boolean needsScores;
|
private final boolean needsScores;
|
||||||
private transient TermContext states[];
|
private transient TermContext states[];
|
||||||
|
|
||||||
public PhraseWeight(IndexSearcher searcher, int postingsFlags)
|
public PhraseWeight(IndexSearcher searcher, boolean needsScores)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
super(PhraseQuery.this);
|
super(PhraseQuery.this);
|
||||||
this.postingsFlags = postingsFlags | PostingsEnum.FLAG_POSITIONS;
|
this.needsScores = needsScores;
|
||||||
this.needsScores = (postingsFlags & PostingsEnum.FLAG_FREQS) != 0;
|
|
||||||
this.similarity = searcher.getSimilarity();
|
this.similarity = searcher.getSimilarity();
|
||||||
final IndexReaderContext context = searcher.getTopReaderContext();
|
final IndexReaderContext context = searcher.getTopReaderContext();
|
||||||
states = new TermContext[terms.size()];
|
states = new TermContext[terms.size()];
|
||||||
|
@ -268,7 +266,7 @@ public class PhraseQuery extends Query {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
te.seekExact(t.bytes(), state);
|
te.seekExact(t.bytes(), state);
|
||||||
PostingsEnum postingsEnum = te.postings(liveDocs, null, postingsFlags);
|
PostingsEnum postingsEnum = te.postings(liveDocs, null, PostingsEnum.FLAG_POSITIONS);
|
||||||
|
|
||||||
// PhraseQuery on a field that did not index
|
// PhraseQuery on a field that did not index
|
||||||
// positions.
|
// positions.
|
||||||
|
@ -320,8 +318,8 @@ public class PhraseQuery extends Query {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Weight createWeight(IndexSearcher searcher, int postingsFlags) throws IOException {
|
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
|
||||||
return new PhraseWeight(searcher, postingsFlags);
|
return new PhraseWeight(searcher, needsScores);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -21,7 +21,6 @@ import java.io.IOException;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
|
||||||
import org.apache.lucene.index.IndexReader;
|
import org.apache.lucene.index.IndexReader;
|
||||||
import org.apache.lucene.index.PostingsEnum;
|
|
||||||
import org.apache.lucene.index.Term;
|
import org.apache.lucene.index.Term;
|
||||||
|
|
||||||
/** The abstract base class for queries.
|
/** The abstract base class for queries.
|
||||||
|
@ -75,10 +74,10 @@ public abstract class Query implements Cloneable {
|
||||||
* <p>
|
* <p>
|
||||||
* Only implemented by primitive queries, which re-write to themselves.
|
* Only implemented by primitive queries, which re-write to themselves.
|
||||||
*
|
*
|
||||||
* @param postingsFlags Bitmask indicating which postings features should be returned
|
* @param needsScores True if document scores ({@link Scorer#score}) or match
|
||||||
* by this query (see {@link PostingsEnum})
|
* frequencies ({@link Scorer#freq}) are needed.
|
||||||
*/
|
*/
|
||||||
public Weight createWeight(IndexSearcher searcher, int postingsFlags) throws IOException {
|
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
|
||||||
throw new UnsupportedOperationException("Query " + this + " does not implement createWeight");
|
throw new UnsupportedOperationException("Query " + this + " does not implement createWeight");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -61,7 +61,7 @@ public abstract class QueryRescorer extends Rescorer {
|
||||||
|
|
||||||
List<LeafReaderContext> leaves = searcher.getIndexReader().leaves();
|
List<LeafReaderContext> leaves = searcher.getIndexReader().leaves();
|
||||||
|
|
||||||
Weight weight = searcher.createNormalizedWeight(query, PostingsEnum.FLAG_FREQS);
|
Weight weight = searcher.createNormalizedWeight(query, true);
|
||||||
|
|
||||||
// Now merge sort docIDs from hits, with reader's leaves:
|
// Now merge sort docIDs from hits, with reader's leaves:
|
||||||
int hitUpto = 0;
|
int hitUpto = 0;
|
||||||
|
|
|
@ -53,7 +53,7 @@ public class QueryWrapperFilter extends Filter {
|
||||||
public DocIdSet getDocIdSet(final LeafReaderContext context, final Bits acceptDocs) throws IOException {
|
public DocIdSet getDocIdSet(final LeafReaderContext context, final Bits acceptDocs) throws IOException {
|
||||||
// get a private context that is used to rewrite, createWeight and score eventually
|
// get a private context that is used to rewrite, createWeight and score eventually
|
||||||
final LeafReaderContext privateContext = context.reader().getContext();
|
final LeafReaderContext privateContext = context.reader().getContext();
|
||||||
final Weight weight = new IndexSearcher(privateContext).createNormalizedWeight(query, PostingsEnum.FLAG_NONE);
|
final Weight weight = new IndexSearcher(privateContext).createNormalizedWeight(query, false);
|
||||||
return new DocIdSet() {
|
return new DocIdSet() {
|
||||||
@Override
|
@Override
|
||||||
public DocIdSetIterator iterator() throws IOException {
|
public DocIdSetIterator iterator() throws IOException {
|
||||||
|
|
|
@ -47,12 +47,12 @@ public class TermQuery extends Query {
|
||||||
private final Similarity similarity;
|
private final Similarity similarity;
|
||||||
private final Similarity.SimWeight stats;
|
private final Similarity.SimWeight stats;
|
||||||
private final TermContext termStates;
|
private final TermContext termStates;
|
||||||
private final int postingsFlags;
|
private final boolean needsScores;
|
||||||
|
|
||||||
public TermWeight(IndexSearcher searcher, int postingsFlags, TermContext termStates)
|
public TermWeight(IndexSearcher searcher, boolean needsScores, TermContext termStates)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
super(TermQuery.this);
|
super(TermQuery.this);
|
||||||
this.postingsFlags = postingsFlags;
|
this.needsScores = needsScores;
|
||||||
assert termStates != null : "TermContext must not be null";
|
assert termStates != null : "TermContext must not be null";
|
||||||
this.termStates = termStates;
|
this.termStates = termStates;
|
||||||
this.similarity = searcher.getSimilarity();
|
this.similarity = searcher.getSimilarity();
|
||||||
|
@ -83,7 +83,7 @@ public class TermQuery extends Query {
|
||||||
if (termsEnum == null) {
|
if (termsEnum == null) {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
PostingsEnum docs = termsEnum.postings(acceptDocs, null, postingsFlags);
|
PostingsEnum docs = termsEnum.postings(acceptDocs, null, needsScores ? PostingsEnum.FLAG_FREQS : PostingsEnum.FLAG_NONE);
|
||||||
assert docs != null;
|
assert docs != null;
|
||||||
return new TermScorer(this, docs, similarity.simScorer(stats, context));
|
return new TermScorer(this, docs, similarity.simScorer(stats, context));
|
||||||
}
|
}
|
||||||
|
@ -168,7 +168,7 @@ public class TermQuery extends Query {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Weight createWeight(IndexSearcher searcher, int postingsFlags) throws IOException {
|
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
|
||||||
final IndexReaderContext context = searcher.getTopReaderContext();
|
final IndexReaderContext context = searcher.getTopReaderContext();
|
||||||
final TermContext termState;
|
final TermContext termState;
|
||||||
if (perReaderTermState == null
|
if (perReaderTermState == null
|
||||||
|
@ -184,7 +184,7 @@ public class TermQuery extends Query {
|
||||||
// we must not ignore the given docFreq - if set use the given value (lie)
|
// we must not ignore the given docFreq - if set use the given value (lie)
|
||||||
if (docFreq != -1) termState.setDocFreq(docFreq);
|
if (docFreq != -1) termState.setDocFreq(docFreq);
|
||||||
|
|
||||||
return new TermWeight(searcher, postingsFlags, termState);
|
return new TermWeight(searcher, needsScores, termState);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -26,7 +26,6 @@ import org.apache.lucene.util.BytesRef;
|
||||||
/** Expert: A <code>Scorer</code> for documents matching a <code>Term</code>.
|
/** Expert: A <code>Scorer</code> for documents matching a <code>Term</code>.
|
||||||
*/
|
*/
|
||||||
final class TermScorer extends Scorer {
|
final class TermScorer extends Scorer {
|
||||||
|
|
||||||
private final PostingsEnum postingsEnum;
|
private final PostingsEnum postingsEnum;
|
||||||
private final Similarity.SimScorer docScorer;
|
private final Similarity.SimScorer docScorer;
|
||||||
|
|
||||||
|
|
|
@ -41,7 +41,7 @@ import org.apache.lucene.util.Bits;
|
||||||
* A <code>Weight</code> is used in the following way:
|
* A <code>Weight</code> is used in the following way:
|
||||||
* <ol>
|
* <ol>
|
||||||
* <li>A <code>Weight</code> is constructed by a top-level query, given a
|
* <li>A <code>Weight</code> is constructed by a top-level query, given a
|
||||||
* <code>IndexSearcher</code> ({@link Query#createWeight(IndexSearcher, int)}).
|
* <code>IndexSearcher</code> ({@link Query#createWeight(IndexSearcher, boolean)}).
|
||||||
* <li>The {@link #getValueForNormalization()} method is called on the
|
* <li>The {@link #getValueForNormalization()} method is called on the
|
||||||
* <code>Weight</code> to compute the query normalization factor
|
* <code>Weight</code> to compute the query normalization factor
|
||||||
* {@link Similarity#queryNorm(float)} of the query clauses contained in the
|
* {@link Similarity#queryNorm(float)} of the query clauses contained in the
|
||||||
|
|
|
@ -358,7 +358,7 @@
|
||||||
* {@link org.apache.lucene.search.Query Query} class has several methods that are important for
|
* {@link org.apache.lucene.search.Query Query} class has several methods that are important for
|
||||||
* derived classes:
|
* derived classes:
|
||||||
* <ol>
|
* <ol>
|
||||||
* <li>{@link org.apache.lucene.search.Query#createWeight(IndexSearcher,int) createWeight(IndexSearcher searcher,boolean)} — A
|
* <li>{@link org.apache.lucene.search.Query#createWeight(IndexSearcher,boolean) createWeight(IndexSearcher searcher,boolean)} — A
|
||||||
* {@link org.apache.lucene.search.Weight Weight} is the internal representation of the
|
* {@link org.apache.lucene.search.Weight Weight} is the internal representation of the
|
||||||
* Query, so each Query implementation must
|
* Query, so each Query implementation must
|
||||||
* provide an implementation of Weight. See the subsection on <a
|
* provide an implementation of Weight. See the subsection on <a
|
||||||
|
@ -367,7 +367,7 @@
|
||||||
* <li>{@link org.apache.lucene.search.Query#rewrite(org.apache.lucene.index.IndexReader) rewrite(IndexReader reader)} — Rewrites queries into primitive queries. Primitive queries are:
|
* <li>{@link org.apache.lucene.search.Query#rewrite(org.apache.lucene.index.IndexReader) rewrite(IndexReader reader)} — Rewrites queries into primitive queries. Primitive queries are:
|
||||||
* {@link org.apache.lucene.search.TermQuery TermQuery},
|
* {@link org.apache.lucene.search.TermQuery TermQuery},
|
||||||
* {@link org.apache.lucene.search.BooleanQuery BooleanQuery}, <span
|
* {@link org.apache.lucene.search.BooleanQuery BooleanQuery}, <span
|
||||||
* >and other queries that implement {@link org.apache.lucene.search.Query#createWeight(IndexSearcher,int) createWeight(IndexSearcher searcher,boolean)}</span></li>
|
* >and other queries that implement {@link org.apache.lucene.search.Query#createWeight(IndexSearcher,boolean) createWeight(IndexSearcher searcher,boolean)}</span></li>
|
||||||
* </ol>
|
* </ol>
|
||||||
* <a name="weightClass"></a>
|
* <a name="weightClass"></a>
|
||||||
* <h3>The Weight Interface</h3>
|
* <h3>The Weight Interface</h3>
|
||||||
|
@ -509,7 +509,7 @@
|
||||||
* <p>Assuming we are not sorting (since sorting doesn't affect the raw Lucene score),
|
* <p>Assuming we are not sorting (since sorting doesn't affect the raw Lucene score),
|
||||||
* we call one of the search methods of the IndexSearcher, passing in the
|
* we call one of the search methods of the IndexSearcher, passing in the
|
||||||
* {@link org.apache.lucene.search.Weight Weight} object created by
|
* {@link org.apache.lucene.search.Weight Weight} object created by
|
||||||
* {@link org.apache.lucene.search.IndexSearcher#createNormalizedWeight(org.apache.lucene.search.Query,int)
|
* {@link org.apache.lucene.search.IndexSearcher#createNormalizedWeight(org.apache.lucene.search.Query,boolean)
|
||||||
* IndexSearcher.createNormalizedWeight(Query,boolean)},
|
* IndexSearcher.createNormalizedWeight(Query,boolean)},
|
||||||
* {@link org.apache.lucene.search.Filter Filter} and the number of results we want.
|
* {@link org.apache.lucene.search.Filter Filter} and the number of results we want.
|
||||||
* This method returns a {@link org.apache.lucene.search.TopDocs TopDocs} object,
|
* This method returns a {@link org.apache.lucene.search.TopDocs TopDocs} object,
|
||||||
|
|
|
@ -71,7 +71,7 @@ public class PayloadNearQuery extends SpanNearQuery {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Weight createWeight(IndexSearcher searcher, int postingsFlags) throws IOException {
|
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
|
||||||
return new PayloadNearSpanWeight(this, searcher);
|
return new PayloadNearSpanWeight(this, searcher);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -67,7 +67,7 @@ public class PayloadTermQuery extends SpanTermQuery {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Weight createWeight(IndexSearcher searcher, int postingsFlags) throws IOException {
|
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
|
||||||
return new PayloadTermWeight(this, searcher);
|
return new PayloadTermWeight(this, searcher);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -106,8 +106,8 @@ public class FieldMaskingSpanQuery extends SpanQuery {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Weight createWeight(IndexSearcher searcher, int postingsFlags) throws IOException {
|
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
|
||||||
return maskedQuery.createWeight(searcher, postingsFlags);
|
return maskedQuery.createWeight(searcher, needsScores);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -42,7 +42,7 @@ public abstract class SpanQuery extends Query {
|
||||||
public abstract String getField();
|
public abstract String getField();
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Weight createWeight(IndexSearcher searcher, int postingsFlags) throws IOException {
|
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
|
||||||
return new SpanWeight(this, searcher);
|
return new SpanWeight(this, searcher);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -264,11 +264,6 @@ final class JustCompileSearch {
|
||||||
throw new UnsupportedOperationException( UNSUPPORTED_MSG );
|
throw new UnsupportedOperationException( UNSUPPORTED_MSG );
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean needsScores() {
|
|
||||||
throw new UnsupportedOperationException(UNSUPPORTED_MSG);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public TopDocs topDocs() {
|
public TopDocs topDocs() {
|
||||||
throw new UnsupportedOperationException( UNSUPPORTED_MSG );
|
throw new UnsupportedOperationException( UNSUPPORTED_MSG );
|
||||||
|
@ -283,6 +278,11 @@ final class JustCompileSearch {
|
||||||
public TopDocs topDocs( int start, int end ) {
|
public TopDocs topDocs( int start, int end ) {
|
||||||
throw new UnsupportedOperationException( UNSUPPORTED_MSG );
|
throw new UnsupportedOperationException( UNSUPPORTED_MSG );
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean needsScores() {
|
||||||
|
throw new UnsupportedOperationException( UNSUPPORTED_MSG );
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
static final class JustCompileWeight extends Weight {
|
static final class JustCompileWeight extends Weight {
|
||||||
|
|
|
@ -28,7 +28,6 @@ import org.apache.lucene.index.FieldInvertState;
|
||||||
import org.apache.lucene.index.IndexWriter;
|
import org.apache.lucene.index.IndexWriter;
|
||||||
import org.apache.lucene.index.IndexWriterConfig;
|
import org.apache.lucene.index.IndexWriterConfig;
|
||||||
import org.apache.lucene.index.LeafReaderContext;
|
import org.apache.lucene.index.LeafReaderContext;
|
||||||
import org.apache.lucene.index.PostingsEnum;
|
|
||||||
import org.apache.lucene.index.Term;
|
import org.apache.lucene.index.Term;
|
||||||
import org.apache.lucene.search.similarities.Similarity;
|
import org.apache.lucene.search.similarities.Similarity;
|
||||||
import org.apache.lucene.store.Directory;
|
import org.apache.lucene.store.Directory;
|
||||||
|
@ -707,7 +706,7 @@ public class TestBooleanCoord extends LuceneTestCase {
|
||||||
/** asserts score for our single matching good doc */
|
/** asserts score for our single matching good doc */
|
||||||
private void assertScore(final float expected, Query query) throws Exception {
|
private void assertScore(final float expected, Query query) throws Exception {
|
||||||
// test in-order
|
// test in-order
|
||||||
Weight weight = searcher.createNormalizedWeight(query, PostingsEnum.FLAG_FREQS);
|
Weight weight = searcher.createNormalizedWeight(query, true);
|
||||||
Scorer scorer = weight.scorer(reader.leaves().get(0), null);
|
Scorer scorer = weight.scorer(reader.leaves().get(0), null);
|
||||||
assertTrue(scorer.docID() == -1 || scorer.docID() == DocIdSetIterator.NO_MORE_DOCS);
|
assertTrue(scorer.docID() == -1 || scorer.docID() == DocIdSetIterator.NO_MORE_DOCS);
|
||||||
assertEquals(0, scorer.nextDoc());
|
assertEquals(0, scorer.nextDoc());
|
||||||
|
|
|
@ -185,7 +185,7 @@ public class TestBooleanOr extends LuceneTestCase {
|
||||||
bq.add(new TermQuery(new Term("field", "a")), BooleanClause.Occur.SHOULD);
|
bq.add(new TermQuery(new Term("field", "a")), BooleanClause.Occur.SHOULD);
|
||||||
bq.add(new TermQuery(new Term("field", "a")), BooleanClause.Occur.SHOULD);
|
bq.add(new TermQuery(new Term("field", "a")), BooleanClause.Occur.SHOULD);
|
||||||
|
|
||||||
Weight w = s.createNormalizedWeight(bq, PostingsEnum.FLAG_FREQS);
|
Weight w = s.createNormalizedWeight(bq, true);
|
||||||
|
|
||||||
assertEquals(1, s.getIndexReader().leaves().size());
|
assertEquals(1, s.getIndexReader().leaves().size());
|
||||||
BulkScorer scorer = w.bulkScorer(s.getIndexReader().leaves().get(0), null);
|
BulkScorer scorer = w.bulkScorer(s.getIndexReader().leaves().get(0), null);
|
||||||
|
|
|
@ -34,7 +34,6 @@ import org.apache.lucene.index.IndexReader;
|
||||||
import org.apache.lucene.index.IndexWriter;
|
import org.apache.lucene.index.IndexWriter;
|
||||||
import org.apache.lucene.index.IndexWriterConfig;
|
import org.apache.lucene.index.IndexWriterConfig;
|
||||||
import org.apache.lucene.index.MultiReader;
|
import org.apache.lucene.index.MultiReader;
|
||||||
import org.apache.lucene.index.PostingsEnum;
|
|
||||||
import org.apache.lucene.index.RandomIndexWriter;
|
import org.apache.lucene.index.RandomIndexWriter;
|
||||||
import org.apache.lucene.index.Term;
|
import org.apache.lucene.index.Term;
|
||||||
import org.apache.lucene.search.similarities.DefaultSimilarity;
|
import org.apache.lucene.search.similarities.DefaultSimilarity;
|
||||||
|
@ -232,7 +231,7 @@ public class TestBooleanQuery extends LuceneTestCase {
|
||||||
q.add(new BooleanClause(new TermQuery(new Term("field", term)), BooleanClause.Occur.SHOULD));
|
q.add(new BooleanClause(new TermQuery(new Term("field", term)), BooleanClause.Occur.SHOULD));
|
||||||
}
|
}
|
||||||
|
|
||||||
Weight weight = s.createNormalizedWeight(q, PostingsEnum.FLAG_FREQS);
|
Weight weight = s.createNormalizedWeight(q, true);
|
||||||
|
|
||||||
Scorer scorer = weight.scorer(s.leafContexts.get(0), null);
|
Scorer scorer = weight.scorer(s.leafContexts.get(0), null);
|
||||||
|
|
||||||
|
@ -250,7 +249,7 @@ public class TestBooleanQuery extends LuceneTestCase {
|
||||||
// verify exact match:
|
// verify exact match:
|
||||||
for(int iter2=0;iter2<10;iter2++) {
|
for(int iter2=0;iter2<10;iter2++) {
|
||||||
|
|
||||||
weight = s.createNormalizedWeight(q, PostingsEnum.FLAG_FREQS);
|
weight = s.createNormalizedWeight(q, true);
|
||||||
scorer = weight.scorer(s.leafContexts.get(0), null);
|
scorer = weight.scorer(s.leafContexts.get(0), null);
|
||||||
|
|
||||||
if (VERBOSE) {
|
if (VERBOSE) {
|
||||||
|
|
|
@ -282,8 +282,8 @@ public class TestBooleanQueryVisitSubscorers extends LuceneTestCase {
|
||||||
static class BooleanQuery2 extends BooleanQuery {
|
static class BooleanQuery2 extends BooleanQuery {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Weight createWeight(IndexSearcher searcher, int postingsFlags) throws IOException {
|
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
|
||||||
return new BooleanWeight(this, searcher, postingsFlags, false) {
|
return new BooleanWeight(this, searcher, needsScores, false) {
|
||||||
@Override
|
@Override
|
||||||
public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
|
public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
|
||||||
Scorer scorer = scorer(context, acceptDocs);
|
Scorer scorer = scorer(context, acceptDocs);
|
||||||
|
|
|
@ -70,7 +70,7 @@ public class TestBooleanScorer extends LuceneTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Weight createWeight(IndexSearcher searcher, int postingsFlags) throws IOException {
|
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
|
||||||
return new Weight(CrazyMustUseBulkScorerQuery.this) {
|
return new Weight(CrazyMustUseBulkScorerQuery.this) {
|
||||||
@Override
|
@Override
|
||||||
public Explanation explain(LeafReaderContext context, int doc) {
|
public Explanation explain(LeafReaderContext context, int doc) {
|
||||||
|
|
|
@ -22,7 +22,6 @@ import java.io.IOException;
|
||||||
import org.apache.lucene.document.Document;
|
import org.apache.lucene.document.Document;
|
||||||
import org.apache.lucene.document.Field;
|
import org.apache.lucene.document.Field;
|
||||||
import org.apache.lucene.index.IndexReader;
|
import org.apache.lucene.index.IndexReader;
|
||||||
import org.apache.lucene.index.PostingsEnum;
|
|
||||||
import org.apache.lucene.index.RandomIndexWriter;
|
import org.apache.lucene.index.RandomIndexWriter;
|
||||||
import org.apache.lucene.index.Term;
|
import org.apache.lucene.index.Term;
|
||||||
import org.apache.lucene.search.similarities.DefaultSimilarity;
|
import org.apache.lucene.search.similarities.DefaultSimilarity;
|
||||||
|
|
|
@ -34,7 +34,6 @@ import org.apache.lucene.index.IndexReader;
|
||||||
import org.apache.lucene.index.IndexWriter;
|
import org.apache.lucene.index.IndexWriter;
|
||||||
import org.apache.lucene.index.IndexWriterConfig;
|
import org.apache.lucene.index.IndexWriterConfig;
|
||||||
import org.apache.lucene.index.LeafReaderContext;
|
import org.apache.lucene.index.LeafReaderContext;
|
||||||
import org.apache.lucene.index.PostingsEnum;
|
|
||||||
import org.apache.lucene.index.RandomIndexWriter;
|
import org.apache.lucene.index.RandomIndexWriter;
|
||||||
import org.apache.lucene.index.SlowCompositeReaderWrapper;
|
import org.apache.lucene.index.SlowCompositeReaderWrapper;
|
||||||
import org.apache.lucene.index.StoredDocument;
|
import org.apache.lucene.index.StoredDocument;
|
||||||
|
@ -179,7 +178,7 @@ public class TestDisjunctionMaxQuery extends LuceneTestCase {
|
||||||
|
|
||||||
QueryUtils.check(random(), dq, s);
|
QueryUtils.check(random(), dq, s);
|
||||||
assertTrue(s.getTopReaderContext() instanceof LeafReaderContext);
|
assertTrue(s.getTopReaderContext() instanceof LeafReaderContext);
|
||||||
final Weight dw = s.createNormalizedWeight(dq, PostingsEnum.FLAG_FREQS);
|
final Weight dw = s.createNormalizedWeight(dq, true);
|
||||||
LeafReaderContext context = (LeafReaderContext)s.getTopReaderContext();
|
LeafReaderContext context = (LeafReaderContext)s.getTopReaderContext();
|
||||||
final Scorer ds = dw.scorer(context, context.reader().getLiveDocs());
|
final Scorer ds = dw.scorer(context, context.reader().getLiveDocs());
|
||||||
final boolean skipOk = ds.advance(3) != DocIdSetIterator.NO_MORE_DOCS;
|
final boolean skipOk = ds.advance(3) != DocIdSetIterator.NO_MORE_DOCS;
|
||||||
|
@ -195,7 +194,7 @@ public class TestDisjunctionMaxQuery extends LuceneTestCase {
|
||||||
dq.add(tq("dek", "DOES_NOT_EXIST"));
|
dq.add(tq("dek", "DOES_NOT_EXIST"));
|
||||||
assertTrue(s.getTopReaderContext() instanceof LeafReaderContext);
|
assertTrue(s.getTopReaderContext() instanceof LeafReaderContext);
|
||||||
QueryUtils.check(random(), dq, s);
|
QueryUtils.check(random(), dq, s);
|
||||||
final Weight dw = s.createNormalizedWeight(dq, PostingsEnum.FLAG_FREQS);
|
final Weight dw = s.createNormalizedWeight(dq, true);
|
||||||
LeafReaderContext context = (LeafReaderContext)s.getTopReaderContext();
|
LeafReaderContext context = (LeafReaderContext)s.getTopReaderContext();
|
||||||
final Scorer ds = dw.scorer(context, context.reader().getLiveDocs());
|
final Scorer ds = dw.scorer(context, context.reader().getLiveDocs());
|
||||||
assertTrue("firsttime skipTo found no match",
|
assertTrue("firsttime skipTo found no match",
|
||||||
|
|
|
@ -31,7 +31,6 @@ import org.apache.lucene.document.SortedSetDocValuesField;
|
||||||
import org.apache.lucene.document.StringField;
|
import org.apache.lucene.document.StringField;
|
||||||
import org.apache.lucene.index.DirectoryReader;
|
import org.apache.lucene.index.DirectoryReader;
|
||||||
import org.apache.lucene.index.LeafReader;
|
import org.apache.lucene.index.LeafReader;
|
||||||
import org.apache.lucene.index.PostingsEnum;
|
|
||||||
import org.apache.lucene.index.RandomIndexWriter;
|
import org.apache.lucene.index.RandomIndexWriter;
|
||||||
import org.apache.lucene.index.SortedSetDocValues;
|
import org.apache.lucene.index.SortedSetDocValues;
|
||||||
import org.apache.lucene.index.Term;
|
import org.apache.lucene.index.Term;
|
||||||
|
@ -125,7 +124,7 @@ public class TestMinShouldMatch2 extends LuceneTestCase {
|
||||||
}
|
}
|
||||||
bq.setMinimumNumberShouldMatch(minShouldMatch);
|
bq.setMinimumNumberShouldMatch(minShouldMatch);
|
||||||
|
|
||||||
BooleanWeight weight = (BooleanWeight) searcher.createNormalizedWeight(bq, PostingsEnum.FLAG_FREQS);
|
BooleanWeight weight = (BooleanWeight) searcher.createNormalizedWeight(bq, true);
|
||||||
|
|
||||||
switch (mode) {
|
switch (mode) {
|
||||||
case DOC_VALUES:
|
case DOC_VALUES:
|
||||||
|
|
|
@ -30,7 +30,6 @@ import org.apache.lucene.index.IndexReader;
|
||||||
import org.apache.lucene.index.IndexWriter;
|
import org.apache.lucene.index.IndexWriter;
|
||||||
import org.apache.lucene.index.IndexWriterConfig;
|
import org.apache.lucene.index.IndexWriterConfig;
|
||||||
import org.apache.lucene.index.MultiFields;
|
import org.apache.lucene.index.MultiFields;
|
||||||
import org.apache.lucene.index.PostingsEnum;
|
|
||||||
import org.apache.lucene.index.RandomIndexWriter;
|
import org.apache.lucene.index.RandomIndexWriter;
|
||||||
import org.apache.lucene.index.Term;
|
import org.apache.lucene.index.Term;
|
||||||
import org.apache.lucene.index.TermsEnum;
|
import org.apache.lucene.index.TermsEnum;
|
||||||
|
@ -345,7 +344,7 @@ public class TestMultiPhraseQuery extends LuceneTestCase {
|
||||||
MultiPhraseQuery query = new MultiPhraseQuery();
|
MultiPhraseQuery query = new MultiPhraseQuery();
|
||||||
query.add(new Term[] { new Term("body", "this"), new Term("body", "that") });
|
query.add(new Term[] { new Term("body", "this"), new Term("body", "that") });
|
||||||
query.add(new Term("body", "is"));
|
query.add(new Term("body", "is"));
|
||||||
Weight weight = query.createWeight(searcher, PostingsEnum.FLAG_FREQS);
|
Weight weight = query.createWeight(searcher, true);
|
||||||
assertEquals(10f * 10f, weight.getValueForNormalization(), 0.001f);
|
assertEquals(10f * 10f, weight.getValueForNormalization(), 0.001f);
|
||||||
|
|
||||||
writer.close();
|
writer.close();
|
||||||
|
|
|
@ -25,7 +25,6 @@ import org.apache.lucene.document.Field;
|
||||||
import org.apache.lucene.document.TextField;
|
import org.apache.lucene.document.TextField;
|
||||||
import org.apache.lucene.index.IndexReader;
|
import org.apache.lucene.index.IndexReader;
|
||||||
import org.apache.lucene.index.LeafReaderContext;
|
import org.apache.lucene.index.LeafReaderContext;
|
||||||
import org.apache.lucene.index.PostingsEnum;
|
|
||||||
import org.apache.lucene.index.RandomIndexWriter;
|
import org.apache.lucene.index.RandomIndexWriter;
|
||||||
import org.apache.lucene.index.Term;
|
import org.apache.lucene.index.Term;
|
||||||
import org.apache.lucene.store.Directory;
|
import org.apache.lucene.store.Directory;
|
||||||
|
@ -64,15 +63,15 @@ public class TestNeedsScores extends LuceneTestCase {
|
||||||
Query required = new TermQuery(new Term("field", "this"));
|
Query required = new TermQuery(new Term("field", "this"));
|
||||||
Query prohibited = new TermQuery(new Term("field", "3"));
|
Query prohibited = new TermQuery(new Term("field", "3"));
|
||||||
BooleanQuery bq = new BooleanQuery();
|
BooleanQuery bq = new BooleanQuery();
|
||||||
bq.add(new AssertNeedsScores(required, PostingsEnum.FLAG_FREQS), BooleanClause.Occur.MUST);
|
bq.add(new AssertNeedsScores(required, true), BooleanClause.Occur.MUST);
|
||||||
bq.add(new AssertNeedsScores(prohibited, PostingsEnum.FLAG_NONE), BooleanClause.Occur.MUST_NOT);
|
bq.add(new AssertNeedsScores(prohibited, false), BooleanClause.Occur.MUST_NOT);
|
||||||
assertEquals(4, searcher.search(bq, 5).totalHits); // we exclude 3
|
assertEquals(4, searcher.search(bq, 5).totalHits); // we exclude 3
|
||||||
}
|
}
|
||||||
|
|
||||||
/** nested inside constant score query */
|
/** nested inside constant score query */
|
||||||
public void testConstantScoreQuery() throws Exception {
|
public void testConstantScoreQuery() throws Exception {
|
||||||
Query term = new TermQuery(new Term("field", "this"));
|
Query term = new TermQuery(new Term("field", "this"));
|
||||||
Query constantScore = new ConstantScoreQuery(new AssertNeedsScores(term, PostingsEnum.FLAG_NONE));
|
Query constantScore = new ConstantScoreQuery(new AssertNeedsScores(term, false));
|
||||||
assertEquals(5, searcher.search(constantScore, 5).totalHits);
|
assertEquals(5, searcher.search(constantScore, 5).totalHits);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -80,38 +79,38 @@ public class TestNeedsScores extends LuceneTestCase {
|
||||||
public void testQueryWrapperFilter() throws Exception {
|
public void testQueryWrapperFilter() throws Exception {
|
||||||
Query query = new MatchAllDocsQuery();
|
Query query = new MatchAllDocsQuery();
|
||||||
Query term = new TermQuery(new Term("field", "this"));
|
Query term = new TermQuery(new Term("field", "this"));
|
||||||
Filter filter = new QueryWrapperFilter(new AssertNeedsScores(term, PostingsEnum.FLAG_NONE));
|
Filter filter = new QueryWrapperFilter(new AssertNeedsScores(term, false));
|
||||||
assertEquals(5, searcher.search(query, filter, 5).totalHits);
|
assertEquals(5, searcher.search(query, filter, 5).totalHits);
|
||||||
}
|
}
|
||||||
|
|
||||||
/** when not sorting by score */
|
/** when not sorting by score */
|
||||||
public void testSortByField() throws Exception {
|
public void testSortByField() throws Exception {
|
||||||
Query query = new AssertNeedsScores(new MatchAllDocsQuery(), PostingsEnum.FLAG_NONE);
|
Query query = new AssertNeedsScores(new MatchAllDocsQuery(), false);
|
||||||
assertEquals(5, searcher.search(query, 5, Sort.INDEXORDER).totalHits);
|
assertEquals(5, searcher.search(query, 5, Sort.INDEXORDER).totalHits);
|
||||||
}
|
}
|
||||||
|
|
||||||
/** when sorting by score */
|
/** when sorting by score */
|
||||||
public void testSortByScore() throws Exception {
|
public void testSortByScore() throws Exception {
|
||||||
Query query = new AssertNeedsScores(new MatchAllDocsQuery(), PostingsEnum.FLAG_FREQS);
|
Query query = new AssertNeedsScores(new MatchAllDocsQuery(), true);
|
||||||
assertEquals(5, searcher.search(query, 5, Sort.RELEVANCE).totalHits);
|
assertEquals(5, searcher.search(query, 5, Sort.RELEVANCE).totalHits);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Wraps a query, checking that the postingsFlags param
|
* Wraps a query, checking that the needsScores param
|
||||||
* passed to Weight.scorer is the expected value.
|
* passed to Weight.scorer is the expected value.
|
||||||
*/
|
*/
|
||||||
static class AssertNeedsScores extends Query {
|
static class AssertNeedsScores extends Query {
|
||||||
final Query in;
|
final Query in;
|
||||||
final int value;
|
final boolean value;
|
||||||
|
|
||||||
AssertNeedsScores(Query in, int value) {
|
AssertNeedsScores(Query in, boolean value) {
|
||||||
this.in = in;
|
this.in = in;
|
||||||
this.value = value;
|
this.value = value;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Weight createWeight(IndexSearcher searcher, int postingsFlags) throws IOException {
|
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
|
||||||
final Weight w = in.createWeight(searcher, postingsFlags);
|
final Weight w = in.createWeight(searcher, needsScores);
|
||||||
return new Weight(AssertNeedsScores.this) {
|
return new Weight(AssertNeedsScores.this) {
|
||||||
@Override
|
@Override
|
||||||
public Explanation explain(LeafReaderContext context, int doc) throws IOException {
|
public Explanation explain(LeafReaderContext context, int doc) throws IOException {
|
||||||
|
@ -130,7 +129,7 @@ public class TestNeedsScores extends LuceneTestCase {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
|
public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
|
||||||
assertEquals("query=" + in, value, postingsFlags);
|
assertEquals("query=" + in, value, needsScores);
|
||||||
return w.scorer(context, acceptDocs);
|
return w.scorer(context, acceptDocs);
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
@ -156,7 +155,7 @@ public class TestNeedsScores extends LuceneTestCase {
|
||||||
final int prime = 31;
|
final int prime = 31;
|
||||||
int result = super.hashCode();
|
int result = super.hashCode();
|
||||||
result = prime * result + ((in == null) ? 0 : in.hashCode());
|
result = prime * result + ((in == null) ? 0 : in.hashCode());
|
||||||
result = prime * result + (value * 37);
|
result = prime * result + (value ? 1231 : 1237);
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -21,7 +21,6 @@ import java.io.IOException;
|
||||||
|
|
||||||
import org.apache.lucene.document.Document;
|
import org.apache.lucene.document.Document;
|
||||||
import org.apache.lucene.index.IndexReader;
|
import org.apache.lucene.index.IndexReader;
|
||||||
import org.apache.lucene.index.PostingsEnum;
|
|
||||||
import org.apache.lucene.index.RandomIndexWriter;
|
import org.apache.lucene.index.RandomIndexWriter;
|
||||||
import org.apache.lucene.index.Term;
|
import org.apache.lucene.index.Term;
|
||||||
import org.apache.lucene.store.Directory;
|
import org.apache.lucene.store.Directory;
|
||||||
|
@ -108,7 +107,7 @@ public class TestPositiveScoresOnlyCollector extends LuceneTestCase {
|
||||||
IndexReader ir = writer.getReader();
|
IndexReader ir = writer.getReader();
|
||||||
writer.close();
|
writer.close();
|
||||||
IndexSearcher searcher = newSearcher(ir);
|
IndexSearcher searcher = newSearcher(ir);
|
||||||
Weight fake = new TermQuery(new Term("fake", "weight")).createWeight(searcher, PostingsEnum.FLAG_FREQS);
|
Weight fake = new TermQuery(new Term("fake", "weight")).createWeight(searcher, true);
|
||||||
Scorer s = new SimpleScorer(fake);
|
Scorer s = new SimpleScorer(fake);
|
||||||
TopDocsCollector<ScoreDoc> tdc = TopScoreDocCollector.create(scores.length);
|
TopDocsCollector<ScoreDoc> tdc = TopScoreDocCollector.create(scores.length);
|
||||||
Collector c = new PositiveScoresOnlyCollector(tdc);
|
Collector c = new PositiveScoresOnlyCollector(tdc);
|
||||||
|
|
|
@ -425,7 +425,7 @@ public class TestQueryRescorer extends LuceneTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Weight createWeight(IndexSearcher searcher, int postingsFlags) throws IOException {
|
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
|
||||||
|
|
||||||
return new Weight(FixedScoreQuery.this) {
|
return new Weight(FixedScoreQuery.this) {
|
||||||
|
|
||||||
|
|
|
@ -20,7 +20,6 @@ package org.apache.lucene.search;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
|
||||||
import org.apache.lucene.index.IndexReader;
|
import org.apache.lucene.index.IndexReader;
|
||||||
import org.apache.lucene.index.PostingsEnum;
|
|
||||||
import org.apache.lucene.index.RandomIndexWriter;
|
import org.apache.lucene.index.RandomIndexWriter;
|
||||||
import org.apache.lucene.index.Term;
|
import org.apache.lucene.index.Term;
|
||||||
import org.apache.lucene.store.Directory;
|
import org.apache.lucene.store.Directory;
|
||||||
|
@ -131,7 +130,7 @@ public class TestScoreCachingWrappingScorer extends LuceneTestCase {
|
||||||
IndexReader ir = writer.getReader();
|
IndexReader ir = writer.getReader();
|
||||||
writer.close();
|
writer.close();
|
||||||
IndexSearcher searcher = newSearcher(ir);
|
IndexSearcher searcher = newSearcher(ir);
|
||||||
Weight fake = new TermQuery(new Term("fake", "weight")).createWeight(searcher, PostingsEnum.FLAG_FREQS);
|
Weight fake = new TermQuery(new Term("fake", "weight")).createWeight(searcher, true);
|
||||||
Scorer s = new SimpleScorer(fake);
|
Scorer s = new SimpleScorer(fake);
|
||||||
ScoreCachingCollector scc = new ScoreCachingCollector(scores.length);
|
ScoreCachingCollector scc = new ScoreCachingCollector(scores.length);
|
||||||
scc.setScorer(s);
|
scc.setScorer(s);
|
||||||
|
|
|
@ -1,172 +0,0 @@
|
||||||
package org.apache.lucene.search;
|
|
||||||
|
|
||||||
/*
|
|
||||||
* 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.
|
|
||||||
*/
|
|
||||||
|
|
||||||
import org.apache.lucene.analysis.MockAnalyzer;
|
|
||||||
import org.apache.lucene.document.Document;
|
|
||||||
import org.apache.lucene.document.FieldType;
|
|
||||||
import org.apache.lucene.document.TextField;
|
|
||||||
import org.apache.lucene.index.DirectoryReader;
|
|
||||||
import org.apache.lucene.index.IndexOptions;
|
|
||||||
import org.apache.lucene.index.IndexReader;
|
|
||||||
import org.apache.lucene.index.IndexWriterConfig;
|
|
||||||
import org.apache.lucene.index.LeafReaderContext;
|
|
||||||
import org.apache.lucene.index.PostingsEnum;
|
|
||||||
import org.apache.lucene.index.RandomIndexWriter;
|
|
||||||
import org.apache.lucene.index.SlowCompositeReaderWrapper;
|
|
||||||
import org.apache.lucene.index.Term;
|
|
||||||
import org.apache.lucene.search.IndexSearcher;
|
|
||||||
import org.apache.lucene.search.Scorer;
|
|
||||||
import org.apache.lucene.search.TermQuery;
|
|
||||||
import org.apache.lucene.search.Weight;
|
|
||||||
import org.apache.lucene.search.payloads.PayloadHelper;
|
|
||||||
import org.apache.lucene.search.similarities.DefaultSimilarity;
|
|
||||||
import org.apache.lucene.store.Directory;
|
|
||||||
import org.apache.lucene.util.BytesRef;
|
|
||||||
import org.apache.lucene.util.LuceneTestCase;
|
|
||||||
import org.junit.Test;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
public class TestTermQueryPostings extends LuceneTestCase {
|
|
||||||
|
|
||||||
private static final String FIELD = "f";
|
|
||||||
|
|
||||||
private static final String[] DOC_FIELDS = new String[]{
|
|
||||||
"a b c d",
|
|
||||||
"a a a a",
|
|
||||||
"c d e f",
|
|
||||||
"b d a g"
|
|
||||||
};
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testTermQueryPositions() throws IOException {
|
|
||||||
|
|
||||||
Directory directory = newDirectory();
|
|
||||||
IndexWriterConfig config = newIndexWriterConfig(new MockAnalyzer(random()));
|
|
||||||
RandomIndexWriter writer = new RandomIndexWriter(random(), directory, config);
|
|
||||||
|
|
||||||
for (String content : DOC_FIELDS) {
|
|
||||||
Document doc = new Document();
|
|
||||||
doc.add(newField(FIELD, content, TextField.TYPE_NOT_STORED));
|
|
||||||
writer.addDocument(doc);
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
IndexReader reader = SlowCompositeReaderWrapper.wrap(writer.getReader());
|
|
||||||
IndexSearcher searcher = new IndexSearcher(reader);
|
|
||||||
writer.close();
|
|
||||||
|
|
||||||
TermQuery tq = new TermQuery(new Term(FIELD, "a"));
|
|
||||||
Weight weight = searcher.createNormalizedWeight(tq, PostingsEnum.FLAG_POSITIONS);
|
|
||||||
LeafReaderContext ctx = (LeafReaderContext) searcher.getTopReaderContext();
|
|
||||||
Scorer scorer = weight.scorer(ctx, null);
|
|
||||||
|
|
||||||
assertEquals(scorer.nextDoc(), 0);
|
|
||||||
assertEquals(scorer.nextPosition(), 0);
|
|
||||||
|
|
||||||
assertEquals(scorer.nextDoc(), 1);
|
|
||||||
assertEquals(scorer.nextPosition(), 0);
|
|
||||||
assertEquals(scorer.nextPosition(), 1);
|
|
||||||
assertEquals(scorer.nextPosition(), 2);
|
|
||||||
assertEquals(scorer.nextPosition(), 3);
|
|
||||||
|
|
||||||
assertEquals(scorer.nextDoc(), 3);
|
|
||||||
assertEquals(scorer.nextPosition(), 2);
|
|
||||||
|
|
||||||
reader.close();
|
|
||||||
directory.close();
|
|
||||||
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testTermQueryOffsets() throws IOException {
|
|
||||||
|
|
||||||
Directory directory = newDirectory();
|
|
||||||
IndexWriterConfig config = newIndexWriterConfig(new MockAnalyzer(random()));
|
|
||||||
RandomIndexWriter writer = new RandomIndexWriter(random(), directory, config);
|
|
||||||
|
|
||||||
FieldType fieldType = new FieldType(TextField.TYPE_NOT_STORED);
|
|
||||||
fieldType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
|
|
||||||
|
|
||||||
for (String content : DOC_FIELDS) {
|
|
||||||
Document doc = new Document();
|
|
||||||
doc.add(newField(FIELD, content, fieldType));
|
|
||||||
writer.addDocument(doc);
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
IndexReader reader = SlowCompositeReaderWrapper.wrap(writer.getReader());
|
|
||||||
IndexSearcher searcher = new IndexSearcher(reader);
|
|
||||||
writer.close();
|
|
||||||
|
|
||||||
TermQuery tq = new TermQuery(new Term(FIELD, "a"));
|
|
||||||
Weight weight = searcher.createNormalizedWeight(tq, PostingsEnum.FLAG_OFFSETS);
|
|
||||||
LeafReaderContext ctx = (LeafReaderContext) searcher.getTopReaderContext();
|
|
||||||
Scorer scorer = weight.scorer(ctx, null);
|
|
||||||
|
|
||||||
assertEquals(scorer.nextDoc(), 0);
|
|
||||||
assertEquals(scorer.nextPosition(), 0);
|
|
||||||
assertEquals(scorer.startOffset(), 0);
|
|
||||||
assertEquals(scorer.endOffset(), 1);
|
|
||||||
|
|
||||||
assertEquals(scorer.nextDoc(), 1);
|
|
||||||
assertEquals(scorer.nextPosition(), 0);
|
|
||||||
assertEquals(scorer.startOffset(), 0);
|
|
||||||
assertEquals(scorer.endOffset(), 1);
|
|
||||||
assertEquals(scorer.nextPosition(), 1);
|
|
||||||
assertEquals(scorer.startOffset(), 2);
|
|
||||||
assertEquals(scorer.endOffset(), 3);
|
|
||||||
assertEquals(scorer.nextPosition(), 2);
|
|
||||||
assertEquals(scorer.startOffset(), 4);
|
|
||||||
assertEquals(scorer.endOffset(), 5);
|
|
||||||
assertEquals(scorer.nextPosition(), 3);
|
|
||||||
assertEquals(scorer.startOffset(), 6);
|
|
||||||
assertEquals(scorer.endOffset(), 7);
|
|
||||||
|
|
||||||
assertEquals(scorer.nextDoc(), 3);
|
|
||||||
assertEquals(scorer.nextPosition(), 2);
|
|
||||||
assertEquals(scorer.startOffset(), 4);
|
|
||||||
assertEquals(scorer.endOffset(), 5);
|
|
||||||
|
|
||||||
reader.close();
|
|
||||||
directory.close();
|
|
||||||
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testTermQueryPayloads() throws Exception {
|
|
||||||
|
|
||||||
PayloadHelper helper = new PayloadHelper();
|
|
||||||
IndexSearcher searcher = helper.setUp(random(), new DefaultSimilarity(), 1000);
|
|
||||||
|
|
||||||
TermQuery tq = new TermQuery(new Term(PayloadHelper.FIELD, "seventy"));
|
|
||||||
Weight weight = searcher.createNormalizedWeight(tq, PostingsEnum.FLAG_PAYLOADS);
|
|
||||||
|
|
||||||
for (LeafReaderContext ctx : searcher.leafContexts) {
|
|
||||||
Scorer scorer = weight.scorer(ctx, null);
|
|
||||||
if (scorer.nextDoc() == DocIdSetIterator.NO_MORE_DOCS)
|
|
||||||
continue;
|
|
||||||
scorer.nextPosition();
|
|
||||||
BytesRef payload = scorer.getPayload();
|
|
||||||
assertEquals(payload.length, 1);
|
|
||||||
}
|
|
||||||
|
|
||||||
helper.tearDown();
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -26,7 +26,6 @@ import org.apache.lucene.document.Document;
|
||||||
import org.apache.lucene.document.Field;
|
import org.apache.lucene.document.Field;
|
||||||
import org.apache.lucene.index.IndexReader;
|
import org.apache.lucene.index.IndexReader;
|
||||||
import org.apache.lucene.index.LeafReaderContext;
|
import org.apache.lucene.index.LeafReaderContext;
|
||||||
import org.apache.lucene.index.PostingsEnum;
|
|
||||||
import org.apache.lucene.index.RandomIndexWriter;
|
import org.apache.lucene.index.RandomIndexWriter;
|
||||||
import org.apache.lucene.index.SlowCompositeReaderWrapper;
|
import org.apache.lucene.index.SlowCompositeReaderWrapper;
|
||||||
import org.apache.lucene.index.Term;
|
import org.apache.lucene.index.Term;
|
||||||
|
@ -76,7 +75,7 @@ public class TestTermScorer extends LuceneTestCase {
|
||||||
Term allTerm = new Term(FIELD, "all");
|
Term allTerm = new Term(FIELD, "all");
|
||||||
TermQuery termQuery = new TermQuery(allTerm);
|
TermQuery termQuery = new TermQuery(allTerm);
|
||||||
|
|
||||||
Weight weight = indexSearcher.createNormalizedWeight(termQuery, PostingsEnum.FLAG_FREQS);
|
Weight weight = indexSearcher.createNormalizedWeight(termQuery, true);
|
||||||
assertTrue(indexSearcher.getTopReaderContext() instanceof LeafReaderContext);
|
assertTrue(indexSearcher.getTopReaderContext() instanceof LeafReaderContext);
|
||||||
LeafReaderContext context = (LeafReaderContext)indexSearcher.getTopReaderContext();
|
LeafReaderContext context = (LeafReaderContext)indexSearcher.getTopReaderContext();
|
||||||
BulkScorer ts = weight.bulkScorer(context, context.reader().getLiveDocs());
|
BulkScorer ts = weight.bulkScorer(context, context.reader().getLiveDocs());
|
||||||
|
@ -138,7 +137,7 @@ public class TestTermScorer extends LuceneTestCase {
|
||||||
Term allTerm = new Term(FIELD, "all");
|
Term allTerm = new Term(FIELD, "all");
|
||||||
TermQuery termQuery = new TermQuery(allTerm);
|
TermQuery termQuery = new TermQuery(allTerm);
|
||||||
|
|
||||||
Weight weight = indexSearcher.createNormalizedWeight(termQuery, PostingsEnum.FLAG_FREQS);
|
Weight weight = indexSearcher.createNormalizedWeight(termQuery, true);
|
||||||
assertTrue(indexSearcher.getTopReaderContext() instanceof LeafReaderContext);
|
assertTrue(indexSearcher.getTopReaderContext() instanceof LeafReaderContext);
|
||||||
LeafReaderContext context = (LeafReaderContext) indexSearcher.getTopReaderContext();
|
LeafReaderContext context = (LeafReaderContext) indexSearcher.getTopReaderContext();
|
||||||
Scorer ts = weight.scorer(context, context.reader().getLiveDocs());
|
Scorer ts = weight.scorer(context, context.reader().getLiveDocs());
|
||||||
|
@ -157,7 +156,7 @@ public class TestTermScorer extends LuceneTestCase {
|
||||||
Term allTerm = new Term(FIELD, "all");
|
Term allTerm = new Term(FIELD, "all");
|
||||||
TermQuery termQuery = new TermQuery(allTerm);
|
TermQuery termQuery = new TermQuery(allTerm);
|
||||||
|
|
||||||
Weight weight = indexSearcher.createNormalizedWeight(termQuery, PostingsEnum.FLAG_FREQS);
|
Weight weight = indexSearcher.createNormalizedWeight(termQuery, true);
|
||||||
assertTrue(indexSearcher.getTopReaderContext() instanceof LeafReaderContext);
|
assertTrue(indexSearcher.getTopReaderContext() instanceof LeafReaderContext);
|
||||||
LeafReaderContext context = (LeafReaderContext) indexSearcher.getTopReaderContext();
|
LeafReaderContext context = (LeafReaderContext) indexSearcher.getTopReaderContext();
|
||||||
Scorer ts = weight.scorer(context, context.reader().getLiveDocs());
|
Scorer ts = weight.scorer(context, context.reader().getLiveDocs());
|
||||||
|
|
|
@ -26,7 +26,6 @@ import org.apache.lucene.index.LeafReaderContext;
|
||||||
import org.apache.lucene.index.CompositeReaderContext;
|
import org.apache.lucene.index.CompositeReaderContext;
|
||||||
import org.apache.lucene.index.IndexReader;
|
import org.apache.lucene.index.IndexReader;
|
||||||
import org.apache.lucene.index.IndexReaderContext;
|
import org.apache.lucene.index.IndexReaderContext;
|
||||||
import org.apache.lucene.index.PostingsEnum;
|
|
||||||
import org.apache.lucene.index.RandomIndexWriter;
|
import org.apache.lucene.index.RandomIndexWriter;
|
||||||
import org.apache.lucene.index.ReaderUtil;
|
import org.apache.lucene.index.ReaderUtil;
|
||||||
import org.apache.lucene.index.Term;
|
import org.apache.lucene.index.Term;
|
||||||
|
@ -252,7 +251,7 @@ public class TestTopDocsMerge extends LuceneTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
// ... then all shards:
|
// ... then all shards:
|
||||||
final Weight w = searcher.createNormalizedWeight(query, PostingsEnum.FLAG_FREQS);
|
final Weight w = searcher.createNormalizedWeight(query, true);
|
||||||
|
|
||||||
final TopDocs[] shardHits;
|
final TopDocs[] shardHits;
|
||||||
if (sort == null) {
|
if (sort == null) {
|
||||||
|
|
|
@ -181,7 +181,7 @@ public class TestNearSpansOrdered extends LuceneTestCase {
|
||||||
*/
|
*/
|
||||||
public void testSpanNearScorerSkipTo1() throws Exception {
|
public void testSpanNearScorerSkipTo1() throws Exception {
|
||||||
SpanNearQuery q = makeQuery();
|
SpanNearQuery q = makeQuery();
|
||||||
Weight w = searcher.createNormalizedWeight(q, PostingsEnum.FLAG_FREQS);
|
Weight w = searcher.createNormalizedWeight(q, true);
|
||||||
IndexReaderContext topReaderContext = searcher.getTopReaderContext();
|
IndexReaderContext topReaderContext = searcher.getTopReaderContext();
|
||||||
LeafReaderContext leave = topReaderContext.leaves().get(0);
|
LeafReaderContext leave = topReaderContext.leaves().get(0);
|
||||||
Scorer s = w.scorer(leave, leave.reader().getLiveDocs());
|
Scorer s = w.scorer(leave, leave.reader().getLiveDocs());
|
||||||
|
|
|
@ -430,7 +430,7 @@ public class TestSpans extends LuceneTestCase {
|
||||||
slop,
|
slop,
|
||||||
ordered);
|
ordered);
|
||||||
|
|
||||||
spanScorer = searcher.createNormalizedWeight(snq, PostingsEnum.FLAG_FREQS).scorer(ctx, ctx.reader().getLiveDocs());
|
spanScorer = searcher.createNormalizedWeight(snq, true).scorer(ctx, ctx.reader().getLiveDocs());
|
||||||
} finally {
|
} finally {
|
||||||
searcher.setSimilarity(oldSim);
|
searcher.setSimilarity(oldSim);
|
||||||
}
|
}
|
||||||
|
|
|
@ -73,8 +73,8 @@ class DrillSidewaysQuery extends Query {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Weight createWeight(IndexSearcher searcher, int postingsFlags) throws IOException {
|
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
|
||||||
final Weight baseWeight = baseQuery.createWeight(searcher, postingsFlags);
|
final Weight baseWeight = baseQuery.createWeight(searcher, needsScores);
|
||||||
final Object[] drillDowns = new Object[drillDownQueries.length];
|
final Object[] drillDowns = new Object[drillDownQueries.length];
|
||||||
for(int dim=0;dim<drillDownQueries.length;dim++) {
|
for(int dim=0;dim<drillDownQueries.length;dim++) {
|
||||||
Query query = drillDownQueries[dim];
|
Query query = drillDownQueries[dim];
|
||||||
|
@ -84,7 +84,7 @@ class DrillSidewaysQuery extends Query {
|
||||||
} else {
|
} else {
|
||||||
// TODO: would be nice if we could say "we will do no
|
// TODO: would be nice if we could say "we will do no
|
||||||
// scoring" here....
|
// scoring" here....
|
||||||
drillDowns[dim] = searcher.rewrite(query).createWeight(searcher, postingsFlags);
|
drillDowns[dim] = searcher.rewrite(query).createWeight(searcher, needsScores);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -44,7 +44,6 @@ import org.apache.lucene.index.IndexReaderContext;
|
||||||
import org.apache.lucene.index.LeafReaderContext;
|
import org.apache.lucene.index.LeafReaderContext;
|
||||||
import org.apache.lucene.index.MultiDocValues;
|
import org.apache.lucene.index.MultiDocValues;
|
||||||
import org.apache.lucene.index.NumericDocValues;
|
import org.apache.lucene.index.NumericDocValues;
|
||||||
import org.apache.lucene.index.PostingsEnum;
|
|
||||||
import org.apache.lucene.index.RandomIndexWriter;
|
import org.apache.lucene.index.RandomIndexWriter;
|
||||||
import org.apache.lucene.index.ReaderUtil;
|
import org.apache.lucene.index.ReaderUtil;
|
||||||
import org.apache.lucene.index.SlowCompositeReaderWrapper;
|
import org.apache.lucene.index.SlowCompositeReaderWrapper;
|
||||||
|
@ -1176,7 +1175,7 @@ public class TestGrouping extends LuceneTestCase {
|
||||||
System.out.println("TEST: " + subSearchers.length + " shards: " + Arrays.toString(subSearchers) + " canUseIDV=" + canUseIDV);
|
System.out.println("TEST: " + subSearchers.length + " shards: " + Arrays.toString(subSearchers) + " canUseIDV=" + canUseIDV);
|
||||||
}
|
}
|
||||||
// Run 1st pass collector to get top groups per shard
|
// Run 1st pass collector to get top groups per shard
|
||||||
final Weight w = topSearcher.createNormalizedWeight(query, PostingsEnum.FLAG_FREQS);
|
final Weight w = topSearcher.createNormalizedWeight(query, true);
|
||||||
final List<Collection<SearchGroup<BytesRef>>> shardGroups = new ArrayList<>();
|
final List<Collection<SearchGroup<BytesRef>>> shardGroups = new ArrayList<>();
|
||||||
List<AbstractFirstPassGroupingCollector<?>> firstPassGroupingCollectors = new ArrayList<>();
|
List<AbstractFirstPassGroupingCollector<?>> firstPassGroupingCollectors = new ArrayList<>();
|
||||||
AbstractFirstPassGroupingCollector<?> firstPassCollector = null;
|
AbstractFirstPassGroupingCollector<?> firstPassCollector = null;
|
||||||
|
|
|
@ -123,8 +123,8 @@ class TermsIncludingScoreQuery extends Query {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Weight createWeight(IndexSearcher searcher, int postingsFlags) throws IOException {
|
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
|
||||||
final Weight originalWeight = originalQuery.createWeight(searcher, postingsFlags);
|
final Weight originalWeight = originalQuery.createWeight(searcher, needsScores);
|
||||||
return new Weight(TermsIncludingScoreQuery.this) {
|
return new Weight(TermsIncludingScoreQuery.this) {
|
||||||
|
|
||||||
private TermsEnum segmentTermsEnum;
|
private TermsEnum segmentTermsEnum;
|
||||||
|
|
|
@ -24,7 +24,6 @@ import java.util.Set;
|
||||||
|
|
||||||
import org.apache.lucene.index.IndexReader;
|
import org.apache.lucene.index.IndexReader;
|
||||||
import org.apache.lucene.index.LeafReaderContext;
|
import org.apache.lucene.index.LeafReaderContext;
|
||||||
import org.apache.lucene.index.PostingsEnum;
|
|
||||||
import org.apache.lucene.index.Term;
|
import org.apache.lucene.index.Term;
|
||||||
import org.apache.lucene.search.Explanation;
|
import org.apache.lucene.search.Explanation;
|
||||||
import org.apache.lucene.search.IndexSearcher;
|
import org.apache.lucene.search.IndexSearcher;
|
||||||
|
@ -84,9 +83,8 @@ public class ToChildBlockJoinQuery extends Query {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Weight createWeight(IndexSearcher searcher, int postingsFlags) throws IOException {
|
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
|
||||||
boolean needsScores = (postingsFlags & PostingsEnum.FLAG_FREQS) != 0;
|
return new ToChildBlockJoinWeight(this, parentQuery.createWeight(searcher, needsScores), parentsFilter, needsScores);
|
||||||
return new ToChildBlockJoinWeight(this, parentQuery.createWeight(searcher, postingsFlags), parentsFilter, needsScores);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Return our parent query. */
|
/** Return our parent query. */
|
||||||
|
|
|
@ -120,8 +120,8 @@ public class ToParentBlockJoinQuery extends Query {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Weight createWeight(IndexSearcher searcher, int postingsFlags) throws IOException {
|
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
|
||||||
return new BlockJoinWeight(this, childQuery.createWeight(searcher, postingsFlags), parentsFilter, scoreMode);
|
return new BlockJoinWeight(this, childQuery.createWeight(searcher, needsScores), parentsFilter, scoreMode);
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Return our child query. */
|
/** Return our child query. */
|
||||||
|
|
|
@ -1188,7 +1188,7 @@ public class TestBlockJoin extends LuceneTestCase {
|
||||||
new TermQuery(new Term("parent", "1"))));
|
new TermQuery(new Term("parent", "1"))));
|
||||||
|
|
||||||
ToParentBlockJoinQuery q = new ToParentBlockJoinQuery(tq, parentFilter, ScoreMode.Avg);
|
ToParentBlockJoinQuery q = new ToParentBlockJoinQuery(tq, parentFilter, ScoreMode.Avg);
|
||||||
Weight weight = s.createNormalizedWeight(q, PostingsEnum.FLAG_FREQS);
|
Weight weight = s.createNormalizedWeight(q, true);
|
||||||
DocIdSetIterator disi = weight.scorer(s.getIndexReader().leaves().get(0), null);
|
DocIdSetIterator disi = weight.scorer(s.getIndexReader().leaves().get(0), null);
|
||||||
assertEquals(1, disi.advance(1));
|
assertEquals(1, disi.advance(1));
|
||||||
r.close();
|
r.close();
|
||||||
|
@ -1222,7 +1222,7 @@ public class TestBlockJoin extends LuceneTestCase {
|
||||||
new TermQuery(new Term("isparent", "yes"))));
|
new TermQuery(new Term("isparent", "yes"))));
|
||||||
|
|
||||||
ToParentBlockJoinQuery q = new ToParentBlockJoinQuery(tq, parentFilter, ScoreMode.Avg);
|
ToParentBlockJoinQuery q = new ToParentBlockJoinQuery(tq, parentFilter, ScoreMode.Avg);
|
||||||
Weight weight = s.createNormalizedWeight(q, PostingsEnum.FLAG_FREQS);
|
Weight weight = s.createNormalizedWeight(q, true);
|
||||||
DocIdSetIterator disi = weight.scorer(s.getIndexReader().leaves().get(0), null);
|
DocIdSetIterator disi = weight.scorer(s.getIndexReader().leaves().get(0), null);
|
||||||
assertEquals(2, disi.advance(0));
|
assertEquals(2, disi.advance(0));
|
||||||
r.close();
|
r.close();
|
||||||
|
|
|
@ -54,8 +54,8 @@ public class BoostingQuery extends Query {
|
||||||
public Query rewrite(IndexReader reader) throws IOException {
|
public Query rewrite(IndexReader reader) throws IOException {
|
||||||
BooleanQuery result = new BooleanQuery() {
|
BooleanQuery result = new BooleanQuery() {
|
||||||
@Override
|
@Override
|
||||||
public Weight createWeight(IndexSearcher searcher, int postingsFlags) throws IOException {
|
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
|
||||||
return new BooleanWeight(this, searcher, postingsFlags, false) {
|
return new BooleanWeight(this, searcher, needsScores, false) {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public float coord(int overlap, int max) {
|
public float coord(int overlap, int max) {
|
||||||
|
|
|
@ -187,12 +187,12 @@ public class CustomScoreQuery extends Query {
|
||||||
boolean qStrict;
|
boolean qStrict;
|
||||||
float queryWeight;
|
float queryWeight;
|
||||||
|
|
||||||
public CustomWeight(IndexSearcher searcher, int postingsFlags) throws IOException {
|
public CustomWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
|
||||||
super(CustomScoreQuery.this);
|
super(CustomScoreQuery.this);
|
||||||
this.subQueryWeight = subQuery.createWeight(searcher, postingsFlags);
|
this.subQueryWeight = subQuery.createWeight(searcher, needsScores);
|
||||||
this.valSrcWeights = new Weight[scoringQueries.length];
|
this.valSrcWeights = new Weight[scoringQueries.length];
|
||||||
for(int i = 0; i < scoringQueries.length; i++) {
|
for(int i = 0; i < scoringQueries.length; i++) {
|
||||||
this.valSrcWeights[i] = scoringQueries[i].createWeight(searcher, postingsFlags);
|
this.valSrcWeights[i] = scoringQueries[i].createWeight(searcher, needsScores);
|
||||||
}
|
}
|
||||||
this.qStrict = strict;
|
this.qStrict = strict;
|
||||||
}
|
}
|
||||||
|
@ -368,8 +368,8 @@ public class CustomScoreQuery extends Query {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Weight createWeight(IndexSearcher searcher, int postingsFlags) throws IOException {
|
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
|
||||||
return new CustomWeight(searcher, postingsFlags);
|
return new CustomWeight(searcher, needsScores);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -68,8 +68,8 @@ public class BoostedQuery extends Query {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Weight createWeight(IndexSearcher searcher, int postingsFlags) throws IOException {
|
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
|
||||||
return new BoostedQuery.BoostedWeight(searcher, postingsFlags);
|
return new BoostedQuery.BoostedWeight(searcher, needsScores);
|
||||||
}
|
}
|
||||||
|
|
||||||
private class BoostedWeight extends Weight {
|
private class BoostedWeight extends Weight {
|
||||||
|
@ -77,10 +77,10 @@ public class BoostedQuery extends Query {
|
||||||
Weight qWeight;
|
Weight qWeight;
|
||||||
Map fcontext;
|
Map fcontext;
|
||||||
|
|
||||||
public BoostedWeight(IndexSearcher searcher, int postingsFlags) throws IOException {
|
public BoostedWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
|
||||||
super(BoostedQuery.this);
|
super(BoostedQuery.this);
|
||||||
this.searcher = searcher;
|
this.searcher = searcher;
|
||||||
this.qWeight = q.createWeight(searcher, postingsFlags);
|
this.qWeight = q.createWeight(searcher, needsScores);
|
||||||
this.fcontext = ValueSource.newContext(searcher);
|
this.fcontext = ValueSource.newContext(searcher);
|
||||||
boostVal.createWeight(fcontext,searcher);
|
boostVal.createWeight(fcontext,searcher);
|
||||||
}
|
}
|
||||||
|
|
|
@ -204,7 +204,7 @@ public class FunctionQuery extends Query {
|
||||||
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Weight createWeight(IndexSearcher searcher, int postingsFlags) throws IOException {
|
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
|
||||||
return new FunctionQuery.FunctionWeight(searcher);
|
return new FunctionQuery.FunctionWeight(searcher);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -21,7 +21,6 @@ import java.io.IOException;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
import org.apache.lucene.index.LeafReaderContext;
|
import org.apache.lucene.index.LeafReaderContext;
|
||||||
import org.apache.lucene.index.PostingsEnum;
|
|
||||||
import org.apache.lucene.index.ReaderUtil;
|
import org.apache.lucene.index.ReaderUtil;
|
||||||
import org.apache.lucene.queries.function.FunctionValues;
|
import org.apache.lucene.queries.function.FunctionValues;
|
||||||
import org.apache.lucene.queries.function.ValueSource;
|
import org.apache.lucene.queries.function.ValueSource;
|
||||||
|
@ -73,7 +72,7 @@ public class QueryValueSource extends ValueSource {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void createWeight(Map context, IndexSearcher searcher) throws IOException {
|
public void createWeight(Map context, IndexSearcher searcher) throws IOException {
|
||||||
Weight w = searcher.createNormalizedWeight(q, PostingsEnum.FLAG_FREQS);
|
Weight w = searcher.createNormalizedWeight(q, true);
|
||||||
context.put(this, w);
|
context.put(this, w);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -188,7 +188,7 @@ public class TermAutomatonQuery extends Query {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Weight createWeight(IndexSearcher searcher, int postingsFlags) throws IOException {
|
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
|
||||||
IndexReaderContext context = searcher.getTopReaderContext();
|
IndexReaderContext context = searcher.getTopReaderContext();
|
||||||
Map<Integer,TermContext> termStates = new HashMap<>();
|
Map<Integer,TermContext> termStates = new HashMap<>();
|
||||||
|
|
||||||
|
|
|
@ -56,8 +56,8 @@ public class AssertingIndexSearcher extends IndexSearcher {
|
||||||
|
|
||||||
/** Ensures, that the returned {@code Weight} is not normalized again, which may produce wrong scores. */
|
/** Ensures, that the returned {@code Weight} is not normalized again, which may produce wrong scores. */
|
||||||
@Override
|
@Override
|
||||||
public Weight createNormalizedWeight(Query query, int postingsFlags) throws IOException {
|
public Weight createNormalizedWeight(Query query, boolean needsScores) throws IOException {
|
||||||
final Weight w = super.createNormalizedWeight(query, postingsFlags);
|
final Weight w = super.createNormalizedWeight(query, needsScores);
|
||||||
return new AssertingWeight(random, w) {
|
return new AssertingWeight(random, w) {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -42,8 +42,8 @@ public class AssertingQuery extends Query {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Weight createWeight(IndexSearcher searcher, int postingsFlags) throws IOException {
|
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
|
||||||
return AssertingWeight.wrap(new Random(random.nextLong()), in.createWeight(searcher, postingsFlags));
|
return AssertingWeight.wrap(new Random(random.nextLong()), in.createWeight(searcher, needsScores));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -32,7 +32,6 @@ import org.apache.lucene.index.IndexWriterConfig;
|
||||||
import org.apache.lucene.index.LeafReader;
|
import org.apache.lucene.index.LeafReader;
|
||||||
import org.apache.lucene.index.LeafReaderContext;
|
import org.apache.lucene.index.LeafReaderContext;
|
||||||
import org.apache.lucene.index.MultiReader;
|
import org.apache.lucene.index.MultiReader;
|
||||||
import org.apache.lucene.index.PostingsEnum;
|
|
||||||
import org.apache.lucene.store.Directory;
|
import org.apache.lucene.store.Directory;
|
||||||
import org.apache.lucene.store.MockDirectoryWrapper;
|
import org.apache.lucene.store.MockDirectoryWrapper;
|
||||||
import org.apache.lucene.store.RAMDirectory;
|
import org.apache.lucene.store.RAMDirectory;
|
||||||
|
@ -262,7 +261,7 @@ public class QueryUtils {
|
||||||
lastDoc[0] = doc;
|
lastDoc[0] = doc;
|
||||||
try {
|
try {
|
||||||
if (scorer == null) {
|
if (scorer == null) {
|
||||||
Weight w = s.createNormalizedWeight(q, PostingsEnum.FLAG_FREQS);
|
Weight w = s.createNormalizedWeight(q, true);
|
||||||
LeafReaderContext context = readerContextArray.get(leafPtr);
|
LeafReaderContext context = readerContextArray.get(leafPtr);
|
||||||
scorer = w.scorer(context, context.reader().getLiveDocs());
|
scorer = w.scorer(context, context.reader().getLiveDocs());
|
||||||
}
|
}
|
||||||
|
@ -314,7 +313,7 @@ public class QueryUtils {
|
||||||
final LeafReader previousReader = lastReader[0];
|
final LeafReader previousReader = lastReader[0];
|
||||||
IndexSearcher indexSearcher = LuceneTestCase.newSearcher(previousReader);
|
IndexSearcher indexSearcher = LuceneTestCase.newSearcher(previousReader);
|
||||||
indexSearcher.setSimilarity(s.getSimilarity());
|
indexSearcher.setSimilarity(s.getSimilarity());
|
||||||
Weight w = indexSearcher.createNormalizedWeight(q, PostingsEnum.FLAG_FREQS);
|
Weight w = indexSearcher.createNormalizedWeight(q, true);
|
||||||
LeafReaderContext ctx = (LeafReaderContext)indexSearcher.getTopReaderContext();
|
LeafReaderContext ctx = (LeafReaderContext)indexSearcher.getTopReaderContext();
|
||||||
Scorer scorer = w.scorer(ctx, ctx.reader().getLiveDocs());
|
Scorer scorer = w.scorer(ctx, ctx.reader().getLiveDocs());
|
||||||
if (scorer != null) {
|
if (scorer != null) {
|
||||||
|
@ -336,7 +335,7 @@ public class QueryUtils {
|
||||||
final LeafReader previousReader = lastReader[0];
|
final LeafReader previousReader = lastReader[0];
|
||||||
IndexSearcher indexSearcher = LuceneTestCase.newSearcher(previousReader, false);
|
IndexSearcher indexSearcher = LuceneTestCase.newSearcher(previousReader, false);
|
||||||
indexSearcher.setSimilarity(s.getSimilarity());
|
indexSearcher.setSimilarity(s.getSimilarity());
|
||||||
Weight w = indexSearcher.createNormalizedWeight(q, PostingsEnum.FLAG_FREQS);
|
Weight w = indexSearcher.createNormalizedWeight(q, true);
|
||||||
LeafReaderContext ctx = previousReader.getContext();
|
LeafReaderContext ctx = previousReader.getContext();
|
||||||
Scorer scorer = w.scorer(ctx, ctx.reader().getLiveDocs());
|
Scorer scorer = w.scorer(ctx, ctx.reader().getLiveDocs());
|
||||||
if (scorer != null) {
|
if (scorer != null) {
|
||||||
|
@ -368,7 +367,7 @@ public class QueryUtils {
|
||||||
try {
|
try {
|
||||||
long startMS = System.currentTimeMillis();
|
long startMS = System.currentTimeMillis();
|
||||||
for (int i=lastDoc[0]+1; i<=doc; i++) {
|
for (int i=lastDoc[0]+1; i<=doc; i++) {
|
||||||
Weight w = s.createNormalizedWeight(q, PostingsEnum.FLAG_FREQS);
|
Weight w = s.createNormalizedWeight(q, true);
|
||||||
Scorer scorer = w.scorer(context.get(leafPtr), liveDocs);
|
Scorer scorer = w.scorer(context.get(leafPtr), liveDocs);
|
||||||
Assert.assertTrue("query collected "+doc+" but skipTo("+i+") says no more docs!",scorer.advance(i) != DocIdSetIterator.NO_MORE_DOCS);
|
Assert.assertTrue("query collected "+doc+" but skipTo("+i+") says no more docs!",scorer.advance(i) != DocIdSetIterator.NO_MORE_DOCS);
|
||||||
Assert.assertEquals("query collected "+doc+" but skipTo("+i+") got to "+scorer.docID(),doc,scorer.docID());
|
Assert.assertEquals("query collected "+doc+" but skipTo("+i+") got to "+scorer.docID(),doc,scorer.docID());
|
||||||
|
@ -401,7 +400,7 @@ public class QueryUtils {
|
||||||
final LeafReader previousReader = lastReader[0];
|
final LeafReader previousReader = lastReader[0];
|
||||||
IndexSearcher indexSearcher = LuceneTestCase.newSearcher(previousReader);
|
IndexSearcher indexSearcher = LuceneTestCase.newSearcher(previousReader);
|
||||||
indexSearcher.setSimilarity(s.getSimilarity());
|
indexSearcher.setSimilarity(s.getSimilarity());
|
||||||
Weight w = indexSearcher.createNormalizedWeight(q, PostingsEnum.FLAG_FREQS);
|
Weight w = indexSearcher.createNormalizedWeight(q, true);
|
||||||
Scorer scorer = w.scorer((LeafReaderContext)indexSearcher.getTopReaderContext(), previousReader.getLiveDocs());
|
Scorer scorer = w.scorer((LeafReaderContext)indexSearcher.getTopReaderContext(), previousReader.getLiveDocs());
|
||||||
if (scorer != null) {
|
if (scorer != null) {
|
||||||
boolean more = scorer.advance(lastDoc[0] + 1) != DocIdSetIterator.NO_MORE_DOCS;
|
boolean more = scorer.advance(lastDoc[0] + 1) != DocIdSetIterator.NO_MORE_DOCS;
|
||||||
|
@ -422,7 +421,7 @@ public class QueryUtils {
|
||||||
final LeafReader previousReader = lastReader[0];
|
final LeafReader previousReader = lastReader[0];
|
||||||
IndexSearcher indexSearcher = LuceneTestCase.newSearcher(previousReader);
|
IndexSearcher indexSearcher = LuceneTestCase.newSearcher(previousReader);
|
||||||
indexSearcher.setSimilarity(s.getSimilarity());
|
indexSearcher.setSimilarity(s.getSimilarity());
|
||||||
Weight w = indexSearcher.createNormalizedWeight(q, PostingsEnum.FLAG_FREQS);
|
Weight w = indexSearcher.createNormalizedWeight(q, true);
|
||||||
Scorer scorer = w.scorer((LeafReaderContext)indexSearcher.getTopReaderContext(), previousReader.getLiveDocs());
|
Scorer scorer = w.scorer((LeafReaderContext)indexSearcher.getTopReaderContext(), previousReader.getLiveDocs());
|
||||||
if (scorer != null) {
|
if (scorer != null) {
|
||||||
boolean more = scorer.advance(lastDoc[0] + 1) != DocIdSetIterator.NO_MORE_DOCS;
|
boolean more = scorer.advance(lastDoc[0] + 1) != DocIdSetIterator.NO_MORE_DOCS;
|
||||||
|
@ -433,7 +432,7 @@ public class QueryUtils {
|
||||||
|
|
||||||
/** Check that the scorer and bulk scorer advance consistently. */
|
/** Check that the scorer and bulk scorer advance consistently. */
|
||||||
public static void checkBulkScorerSkipTo(Random r, Query query, IndexSearcher searcher) throws IOException {
|
public static void checkBulkScorerSkipTo(Random r, Query query, IndexSearcher searcher) throws IOException {
|
||||||
Weight weight = searcher.createNormalizedWeight(query, PostingsEnum.FLAG_FREQS);
|
Weight weight = searcher.createNormalizedWeight(query, true);
|
||||||
for (LeafReaderContext context : searcher.getIndexReader().leaves()) {
|
for (LeafReaderContext context : searcher.getIndexReader().leaves()) {
|
||||||
final Scorer scorer = weight.scorer(context, context.reader().getLiveDocs());
|
final Scorer scorer = weight.scorer(context, context.reader().getLiveDocs());
|
||||||
final BulkScorer bulkScorer = weight.bulkScorer(context, context.reader().getLiveDocs());
|
final BulkScorer bulkScorer = weight.bulkScorer(context, context.reader().getLiveDocs());
|
||||||
|
|
|
@ -563,7 +563,7 @@ class SpatialDistanceQuery extends ExtendedQueryBase implements PostFilter {
|
||||||
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Weight createWeight(IndexSearcher searcher, int postingsFlags) throws IOException {
|
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
|
||||||
// if we were supposed to use bboxQuery, then we should have been rewritten using that query
|
// if we were supposed to use bboxQuery, then we should have been rewritten using that query
|
||||||
assert bboxQuery == null;
|
assert bboxQuery == null;
|
||||||
return new SpatialWeight(searcher);
|
return new SpatialWeight(searcher);
|
||||||
|
|
|
@ -80,7 +80,7 @@ public class ExportQParserPlugin extends QParserPlugin {
|
||||||
}
|
}
|
||||||
|
|
||||||
public Weight createWeight(IndexSearcher searcher) throws IOException {
|
public Weight createWeight(IndexSearcher searcher) throws IOException {
|
||||||
return mainQuery.createWeight(searcher, PostingsEnum.FLAG_FREQS);
|
return mainQuery.createWeight(searcher, true);
|
||||||
}
|
}
|
||||||
|
|
||||||
public Query rewrite(IndexReader reader) throws IOException {
|
public Query rewrite(IndexReader reader) throws IOException {
|
||||||
|
|
|
@ -210,7 +210,7 @@ class JoinQuery extends Query {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Weight createWeight(IndexSearcher searcher, int postingsFlags) throws IOException {
|
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
|
||||||
return new JoinQueryWeight((SolrIndexSearcher)searcher);
|
return new JoinQueryWeight((SolrIndexSearcher)searcher);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -27,7 +27,6 @@ import com.carrotsearch.hppc.IntFloatOpenHashMap;
|
||||||
import com.carrotsearch.hppc.IntIntOpenHashMap;
|
import com.carrotsearch.hppc.IntIntOpenHashMap;
|
||||||
import org.apache.lucene.index.IndexReader;
|
import org.apache.lucene.index.IndexReader;
|
||||||
import org.apache.lucene.index.LeafReaderContext;
|
import org.apache.lucene.index.LeafReaderContext;
|
||||||
import org.apache.lucene.index.PostingsEnum;
|
|
||||||
import org.apache.lucene.index.Term;
|
import org.apache.lucene.index.Term;
|
||||||
import org.apache.lucene.search.Explanation;
|
import org.apache.lucene.search.Explanation;
|
||||||
import org.apache.lucene.search.IndexSearcher;
|
import org.apache.lucene.search.IndexSearcher;
|
||||||
|
@ -172,9 +171,8 @@ public class ReRankQParserPlugin extends QParserPlugin {
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException{
|
||||||
public Weight createWeight(IndexSearcher searcher, int postingsFlags) throws IOException{
|
return new ReRankWeight(mainQuery, reRankQuery, reRankWeight, searcher, needsScores);
|
||||||
return new ReRankWeight(mainQuery, reRankQuery, reRankWeight, searcher, postingsFlags);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -184,12 +182,12 @@ public class ReRankQParserPlugin extends QParserPlugin {
|
||||||
private Weight mainWeight;
|
private Weight mainWeight;
|
||||||
private double reRankWeight;
|
private double reRankWeight;
|
||||||
|
|
||||||
public ReRankWeight(Query mainQuery, Query reRankQuery, double reRankWeight, IndexSearcher searcher, int postingsFlags) throws IOException {
|
public ReRankWeight(Query mainQuery, Query reRankQuery, double reRankWeight, IndexSearcher searcher, boolean needsScores) throws IOException {
|
||||||
super(mainQuery);
|
super(mainQuery);
|
||||||
this.reRankQuery = reRankQuery;
|
this.reRankQuery = reRankQuery;
|
||||||
this.searcher = searcher;
|
this.searcher = searcher;
|
||||||
this.reRankWeight = reRankWeight;
|
this.reRankWeight = reRankWeight;
|
||||||
this.mainWeight = mainQuery.createWeight(searcher, postingsFlags);
|
this.mainWeight = mainQuery.createWeight(searcher, needsScores);
|
||||||
}
|
}
|
||||||
|
|
||||||
public float getValueForNormalization() throws IOException {
|
public float getValueForNormalization() throws IOException {
|
||||||
|
|
|
@ -229,7 +229,7 @@ public class SolrConstantScoreQuery extends ConstantScoreQuery implements Extend
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Weight createWeight(IndexSearcher searcher, int postingsFlags) {
|
public Weight createWeight(IndexSearcher searcher, boolean needsScores) {
|
||||||
try {
|
try {
|
||||||
return new SolrConstantScoreQuery.ConstantWeight(searcher);
|
return new SolrConstantScoreQuery.ConstantWeight(searcher);
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
|
|
|
@ -1113,7 +1113,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable,SolrIn
|
||||||
List<Weight> weights = new ArrayList<>(notCached.size());
|
List<Weight> weights = new ArrayList<>(notCached.size());
|
||||||
for (Query q : notCached) {
|
for (Query q : notCached) {
|
||||||
Query qq = QueryUtils.makeQueryable(q);
|
Query qq = QueryUtils.makeQueryable(q);
|
||||||
weights.add(createNormalizedWeight(qq, PostingsEnum.FLAG_FREQS));
|
weights.add(createNormalizedWeight(qq, true));
|
||||||
}
|
}
|
||||||
pf.filter = new FilterImpl(answer, weights);
|
pf.filter = new FilterImpl(answer, weights);
|
||||||
} else {
|
} else {
|
||||||
|
|
|
@ -53,8 +53,8 @@ public class WrappedQuery extends ExtendedQueryBase {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Weight createWeight(IndexSearcher searcher, int postingsFlags) throws IOException {
|
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
|
||||||
return q.createWeight(searcher, postingsFlags);
|
return q.createWeight(searcher, needsScores);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -53,8 +53,8 @@ public class IgnoreAcceptDocsQuery extends Query {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Weight createWeight(IndexSearcher searcher, int postingsFlags) throws IOException {
|
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
|
||||||
Weight inner = q.createWeight(searcher, postingsFlags);
|
Weight inner = q.createWeight(searcher, needsScores);
|
||||||
return new IADWeight(inner);
|
return new IADWeight(inner);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -64,10 +64,10 @@ final class DeleteByQueryWrapper extends Query {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Weight createWeight(IndexSearcher searcher, int postingsFlags) throws IOException {
|
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
|
||||||
final LeafReader wrapped = wrap((LeafReader) searcher.getIndexReader());
|
final LeafReader wrapped = wrap((LeafReader) searcher.getIndexReader());
|
||||||
final IndexSearcher privateContext = new IndexSearcher(wrapped);
|
final IndexSearcher privateContext = new IndexSearcher(wrapped);
|
||||||
final Weight inner = in.createWeight(privateContext, postingsFlags);
|
final Weight inner = in.createWeight(privateContext, needsScores);
|
||||||
return new Weight(DeleteByQueryWrapper.this) {
|
return new Weight(DeleteByQueryWrapper.this) {
|
||||||
@Override
|
@Override
|
||||||
public Explanation explain(LeafReaderContext context, int doc) throws IOException { throw new UnsupportedOperationException(); }
|
public Explanation explain(LeafReaderContext context, int doc) throws IOException { throw new UnsupportedOperationException(); }
|
||||||
|
|
|
@ -32,7 +32,6 @@ import org.apache.lucene.index.DocValues;
|
||||||
import org.apache.lucene.index.IndexReaderContext;
|
import org.apache.lucene.index.IndexReaderContext;
|
||||||
import org.apache.lucene.index.LeafReaderContext;
|
import org.apache.lucene.index.LeafReaderContext;
|
||||||
import org.apache.lucene.index.NumericDocValues;
|
import org.apache.lucene.index.NumericDocValues;
|
||||||
import org.apache.lucene.index.PostingsEnum;
|
|
||||||
import org.apache.lucene.index.ReaderUtil;
|
import org.apache.lucene.index.ReaderUtil;
|
||||||
import org.apache.lucene.search.FieldComparator;
|
import org.apache.lucene.search.FieldComparator;
|
||||||
import org.apache.lucene.search.IndexSearcher;
|
import org.apache.lucene.search.IndexSearcher;
|
||||||
|
@ -115,9 +114,8 @@ public class TestRankQueryPlugin extends QParserPlugin {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
public Weight createWeight(IndexSearcher indexSearcher, boolean needsScores) throws IOException{
|
||||||
public Weight createWeight(IndexSearcher indexSearcher, int postingsFlags) throws IOException{
|
return q.createWeight(indexSearcher, needsScores);
|
||||||
return q.createWeight(indexSearcher, postingsFlags);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public void setBoost(float boost) {
|
public void setBoost(float boost) {
|
||||||
|
|
Loading…
Reference in New Issue