LUCENE-6179: Disallow out-of-order scoring.

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1652013 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Adrien Grand 2015-01-15 09:35:20 +00:00
parent 8a15a5c544
commit 457e70ab94
98 changed files with 449 additions and 2119 deletions

View File

@ -364,6 +364,10 @@ API Changes
* LUCENE-6158, LUCENE-6165: IndexWriter.addIndexes(IndexReader...) changed to * LUCENE-6158, LUCENE-6165: IndexWriter.addIndexes(IndexReader...) changed to
addIndexes(CodecReader...) (Robert Muir) addIndexes(CodecReader...) (Robert Muir)
* LUCENE-6179: Out-of-order scoring is not allowed anymore, so
Weight.scoresDocsOutOfOrder and LeafCollector.acceptsDocsOutOfOrder have been
removed and boolean queries now always score in order.
Bug Fixes Bug Fixes
* LUCENE-5650: Enforce read-only access to any path outside the temporary * LUCENE-5650: Enforce read-only access to any path outside the temporary

View File

@ -124,8 +124,7 @@ public abstract class ReadTask extends PerfTask {
// pulling the Weight ourselves: // pulling the Weight ourselves:
TopFieldCollector collector = TopFieldCollector.create(sort, numHits, TopFieldCollector collector = TopFieldCollector.create(sort, numHits,
true, withScore(), true, withScore(),
withMaxScore(), withMaxScore());
false);
searcher.search(q, null, collector); searcher.search(q, null, collector);
hits = collector.topDocs(); hits = collector.topDocs();
} else { } else {
@ -191,7 +190,7 @@ public abstract class ReadTask extends PerfTask {
} }
protected Collector createCollector() throws Exception { protected Collector createCollector() throws Exception {
return TopScoreDocCollector.create(numHits(), true); return TopScoreDocCollector.create(numHits());
} }

View File

@ -52,10 +52,8 @@ public class SearchWithCollectorTask extends SearchTask {
@Override @Override
protected Collector createCollector() throws Exception { protected Collector createCollector() throws Exception {
Collector collector = null; Collector collector = null;
if (clnName.equalsIgnoreCase("topScoreDocOrdered") == true) { if (clnName.equalsIgnoreCase("topScoreDoc") == true) {
collector = TopScoreDocCollector.create(numHits(), true); collector = TopScoreDocCollector.create(numHits());
} else if (clnName.equalsIgnoreCase("topScoreDocUnOrdered") == true) {
collector = TopScoreDocCollector.create(numHits(), false);
} else if (clnName.length() > 0){ } else if (clnName.length() > 0){
collector = Class.forName(clnName).asSubclass(Collector.class).newInstance(); collector = Class.forName(clnName).asSubclass(Collector.class).newInstance();

View File

@ -24,8 +24,8 @@ import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Set; import java.util.Set;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.Term; import org.apache.lucene.index.Term;
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;
@ -305,21 +305,19 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
} }
@Override @Override
public BulkScorer bulkScorer(LeafReaderContext context, boolean scoreDocsInOrder, public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
Bits acceptDocs) throws IOException {
if (scoreDocsInOrder || minNrShouldMatch > 1) { if (minNrShouldMatch > 1) {
// TODO: (LUCENE-4872) in some cases BooleanScorer may be faster for minNrShouldMatch // TODO: (LUCENE-4872) in some cases BooleanScorer may be faster for minNrShouldMatch
// but the same is even true of pure conjunctions... // but the same is even true of pure conjunctions...
return super.bulkScorer(context, scoreDocsInOrder, acceptDocs); return super.bulkScorer(context, acceptDocs);
} }
List<BulkScorer> prohibited = new ArrayList<BulkScorer>();
List<BulkScorer> optional = new ArrayList<BulkScorer>(); List<BulkScorer> optional = new ArrayList<BulkScorer>();
Iterator<BooleanClause> cIter = clauses.iterator(); Iterator<BooleanClause> cIter = clauses.iterator();
for (Weight w : weights) { for (Weight w : weights) {
BooleanClause c = cIter.next(); BooleanClause c = cIter.next();
BulkScorer subScorer = w.bulkScorer(context, false, acceptDocs); BulkScorer subScorer = w.bulkScorer(context, acceptDocs);
if (subScorer == null) { if (subScorer == null) {
if (c.isRequired()) { if (c.isRequired()) {
return null; return null;
@ -328,16 +326,20 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
// TODO: there are some cases where BooleanScorer // TODO: there are some cases where BooleanScorer
// would handle conjunctions faster than // would handle conjunctions faster than
// BooleanScorer2... // BooleanScorer2...
return super.bulkScorer(context, scoreDocsInOrder, acceptDocs); return super.bulkScorer(context, acceptDocs);
} else if (c.isProhibited()) { } else if (c.isProhibited()) {
// TODO: there are some cases where BooleanScorer could do this faster // TODO: there are some cases where BooleanScorer could do this faster
return super.bulkScorer(context, scoreDocsInOrder, acceptDocs); return super.bulkScorer(context, acceptDocs);
} else { } else {
optional.add(subScorer); optional.add(subScorer);
} }
} }
return new BooleanScorer(this, disableCoord, minNrShouldMatch, optional, prohibited, maxCoord); if (optional.size() == 0) {
return null;
}
return new BooleanScorer(this, disableCoord, maxCoord, optional);
} }
@Override @Override
@ -432,30 +434,6 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
} }
} }
@Override
public boolean scoresDocsOutOfOrder() {
if (minNrShouldMatch > 1) {
// BS2 (in-order) will be used by scorer()
return false;
}
int optionalCount = 0;
for (BooleanClause c : clauses) {
if (c.isRequired()) {
// BS2 (in-order) will be used by scorer()
return false;
} else if (!c.isProhibited()) {
optionalCount++;
}
}
if (optionalCount == minNrShouldMatch) {
return false; // BS2 (in-order) will be used, as this means conjunction
}
// scorer() will return an out-of-order scorer if requested.
return true;
}
private Scorer req(List<Scorer> required, boolean disableCoord) { private Scorer req(List<Scorer> required, boolean disableCoord) {
if (required.size() == 1) { if (required.size() == 1) {
Scorer req = required.get(0); Scorer req = required.get(0);

View File

@ -18,75 +18,41 @@ package org.apache.lucene.search;
*/ */
import java.io.IOException; import java.io.IOException;
import java.util.List; import java.util.Arrays;
import java.util.Collection;
import org.apache.lucene.search.BooleanQuery.BooleanWeight; import org.apache.lucene.search.BooleanQuery.BooleanWeight;
/* Description from Doug Cutting (excerpted from /**
* LUCENE-1483): * BulkSorer that is used for pure disjunctions: no MUST clauses and
* * minShouldMatch == 1. This scorer scores documents by batches of 2048 docs.
* BooleanScorer uses an array to score windows of */
* 2K docs. So it scores docs 0-2K first, then docs 2K-4K,
* etc. For each window it iterates through all query terms
* and accumulates a score in table[doc%2K]. It also stores
* in the table a bitmask representing which terms
* contributed to the score. Non-zero scores are chained in
* a linked list. At the end of scoring each window it then
* iterates through the linked list and, if the bitmask
* matches the boolean constraints, collects a hit. For
* boolean queries with lots of frequent terms this can be
* much faster, since it does not need to update a priority
* queue for each posting, instead performing constant-time
* operations per posting. The only downside is that it
* results in hits being delivered out-of-order within the
* window, which means it cannot be nested within other
* scorers. But it works well as a top-level scorer.
*
* The new BooleanScorer2 implementation instead works by
* merging priority queues of postings, albeit with some
* clever tricks. For example, a pure conjunction (all terms
* required) does not require a priority queue. Instead it
* sorts the posting streams at the start, then repeatedly
* skips the first to to the last. If the first ever equals
* the last, then there's a hit. When some terms are
* required and some terms are optional, the conjunction can
* be evaluated first, then the optional terms can all skip
* to the match and be added to the score. Thus the
* conjunction can reduce the number of priority queue
* updates for the optional terms. */
final class BooleanScorer extends BulkScorer { final class BooleanScorer extends BulkScorer {
private static final class BooleanScorerCollector extends SimpleCollector { static final int SHIFT = 11;
private BucketTable bucketTable; static final int SIZE = 1 << SHIFT;
private int mask; static final int MASK = SIZE - 1;
private Scorer scorer; static final int SET_SIZE = 1 << (SHIFT - 6);
static final int SET_MASK = SET_SIZE - 1;
public BooleanScorerCollector(int mask, BucketTable bucketTable) { static class Bucket {
this.mask = mask; double score;
this.bucketTable = bucketTable; int freq;
} }
@Override final Bucket[] buckets = new Bucket[SIZE];
public void collect(final int doc) throws IOException { // This is basically an inlined FixedBitSet... seems to help with bound checks
final BucketTable table = bucketTable; final long[] matching = new long[SET_SIZE];
final int i = doc & BucketTable.MASK;
final Bucket bucket = table.buckets[i];
if (bucket.doc != doc) { // invalid bucket final float[] coordFactors;
bucket.doc = doc; // set doc final BulkScorer[] optionalScorers;
bucket.score = scorer.score(); // initialize score final FakeScorer fakeScorer = new FakeScorer();
bucket.bits = mask; // initialize mask
bucket.coord = 1; // initialize coord
bucket.next = table.first; // push onto valid list boolean hasMatches;
table.first = bucket; int max = 0;
} else { // valid bucket
bucket.score += scorer.score(); // increment score final class OrCollector implements LeafCollector {
bucket.bits |= mask; // add bits in mask Scorer scorer;
bucket.coord++; // increment coord
}
}
@Override @Override
public void setScorer(Scorer scorer) { public void setScorer(Scorer scorer) {
@ -94,95 +60,24 @@ final class BooleanScorer extends BulkScorer {
} }
@Override @Override
public boolean acceptsDocsOutOfOrder() { public void collect(int doc) throws IOException {
return true; hasMatches = true;
} final int i = doc & MASK;
final int idx = i >>> 6;
} matching[idx] |= 1L << i;
final Bucket bucket = buckets[i];
static final class Bucket { bucket.freq++;
int doc = -1; // tells if bucket is valid bucket.score += scorer.score();
double score; // incremental score
// TODO: break out bool anyProhibited, int
// numRequiredMatched; then we can remove 32 limit on
// required clauses
int bits; // used for bool constraints
int coord; // count of terms in score
Bucket next; // next valid bucket
}
/** A simple hash table of document scores within a range. */
static final class BucketTable {
public static final int SIZE = 1 << 11;
public static final int MASK = SIZE - 1;
final Bucket[] buckets = new Bucket[SIZE];
Bucket first = null; // head of valid list
public BucketTable() {
// Pre-fill to save the lazy init when collecting
// each sub:
for(int idx=0;idx<SIZE;idx++) {
buckets[idx] = new Bucket();
} }
} }
public LeafCollector newCollector(int mask) { final OrCollector orCollector = new OrCollector();
return new BooleanScorerCollector(mask, this);
} BooleanScorer(BooleanWeight weight, boolean disableCoord, int maxCoord, Collection<BulkScorer> optionalScorers) {
for (int i = 0; i < buckets.length; i++) {
public int size() { return SIZE; } buckets[i] = new Bucket();
}
static final class SubScorer {
public BulkScorer scorer;
// TODO: re-enable this if BQ ever sends us required clauses
//public boolean required = false;
public boolean prohibited;
public LeafCollector collector;
public SubScorer next;
public boolean more;
public SubScorer(BulkScorer scorer, boolean required, boolean prohibited,
LeafCollector collector, SubScorer next) {
if (required) {
throw new IllegalArgumentException("this scorer cannot handle required=true");
}
this.scorer = scorer;
this.more = true;
// TODO: re-enable this if BQ ever sends us required clauses
//this.required = required;
this.prohibited = prohibited;
this.collector = collector;
this.next = next;
}
}
private SubScorer scorers = null;
private BucketTable bucketTable = new BucketTable();
private final float[] coordFactors;
// TODO: re-enable this if BQ ever sends us required clauses
//private int requiredMask = 0;
private final int minNrShouldMatch;
private int end;
private Bucket current;
// Any time a prohibited clause matches we set bit 0:
private static final int PROHIBITED_MASK = 1;
private final Weight weight;
BooleanScorer(BooleanWeight weight, boolean disableCoord, int minNrShouldMatch,
List<BulkScorer> optionalScorers, List<BulkScorer> prohibitedScorers, int maxCoord) throws IOException {
this.minNrShouldMatch = minNrShouldMatch;
this.weight = weight;
for (BulkScorer scorer : optionalScorers) {
scorers = new SubScorer(scorer, false, false, bucketTable.newCollector(0), scorers);
}
for (BulkScorer scorer : prohibitedScorers) {
scorers = new SubScorer(scorer, false, true, bucketTable.newCollector(PROHIBITED_MASK), scorers);
} }
this.optionalScorers = optionalScorers.toArray(new BulkScorer[0]);
coordFactors = new float[optionalScorers.size() + 1]; coordFactors = new float[optionalScorers.size() + 1];
for (int i = 0; i < coordFactors.length; i++) { for (int i = 0; i < coordFactors.length; i++) {
@ -190,83 +85,62 @@ final class BooleanScorer extends BulkScorer {
} }
} }
private void scoreDocument(LeafCollector collector, int base, int i) throws IOException {
final Bucket bucket = buckets[i];
fakeScorer.freq = bucket.freq;
fakeScorer.score = (float) bucket.score * coordFactors[bucket.freq];
final int doc = base | i;
fakeScorer.doc = doc;
collector.collect(doc);
bucket.freq = 0;
bucket.score = 0;
}
private void scoreMatches(LeafCollector collector, int base) throws IOException {
long matching[] = this.matching;
for (int idx = 0; idx < matching.length; idx++) {
long bits = matching[idx];
while (bits != 0L) {
int ntz = Long.numberOfTrailingZeros(bits);
int doc = idx << 6 | ntz;
scoreDocument(collector, base, doc);
bits ^= 1L << ntz;
}
}
}
private boolean collectMatches() throws IOException {
boolean more = false;
for (BulkScorer scorer : optionalScorers) {
more |= scorer.score(orCollector, max);
}
return more;
}
private boolean scoreWindow(LeafCollector collector, int base, int max) throws IOException {
this.max = Math.min(base + SIZE, max);
hasMatches = false;
boolean more = collectMatches();
if (hasMatches) {
scoreMatches(collector, base);
Arrays.fill(matching, 0L);
}
return more;
}
@Override @Override
public boolean score(LeafCollector collector, int max) throws IOException { public boolean score(LeafCollector collector, int max) throws IOException {
fakeScorer.doc = -1;
collector.setScorer(fakeScorer);
boolean more; for (int docBase = this.max & ~MASK; docBase < max; docBase += SIZE) {
Bucket tmp; if (scoreWindow(collector, docBase, max) == false) {
FakeScorer fs = new FakeScorer();
// The internal loop will set the score and doc before calling collect.
collector.setScorer(fs);
do {
bucketTable.first = null;
while (current != null) { // more queued
// check prohibited & required
if ((current.bits & PROHIBITED_MASK) == 0) {
// TODO: re-enable this if BQ ever sends us required
// clauses
//&& (current.bits & requiredMask) == requiredMask) {
// NOTE: Lucene always passes max =
// Integer.MAX_VALUE today, because we never embed
// a BooleanScorer inside another (even though
// that should work)... but in theory an outside
// app could pass a different max so we must check
// it:
if (current.doc >= max) {
tmp = current;
current = current.next;
tmp.next = bucketTable.first;
bucketTable.first = tmp;
continue;
}
if (current.coord >= minNrShouldMatch) {
fs.score = (float) (current.score * coordFactors[current.coord]);
fs.doc = current.doc;
fs.freq = current.coord;
collector.collect(current.doc);
}
}
current = current.next; // pop the queue
}
if (bucketTable.first != null){
current = bucketTable.first;
bucketTable.first = current.next;
return true;
}
// refill the queue
more = false;
end += BucketTable.SIZE;
for (SubScorer sub = scorers; sub != null; sub = sub.next) {
if (sub.more) {
sub.more = sub.scorer.score(sub.collector, end);
more |= sub.more;
}
}
current = bucketTable.first;
} while (current != null || more);
return false; return false;
} }
}
@Override return true;
public String toString() {
StringBuilder buffer = new StringBuilder();
buffer.append("boolean(");
for (SubScorer sub = scorers; sub != null; sub = sub.next) {
buffer.append(sub.scorer.toString());
buffer.append(" ");
}
buffer.append(")");
return buffer.toString();
} }
} }

View File

@ -82,7 +82,6 @@ public abstract class CachingCollector extends FilterCollector {
private static class NoScoreCachingCollector extends CachingCollector { private static class NoScoreCachingCollector extends CachingCollector {
List<Boolean> acceptDocsOutOfOrders;
List<LeafReaderContext> contexts; List<LeafReaderContext> contexts;
List<int[]> docs; List<int[]> docs;
int maxDocsToCache; int maxDocsToCache;
@ -92,7 +91,6 @@ public abstract class CachingCollector extends FilterCollector {
super(in); super(in);
this.maxDocsToCache = maxDocsToCache; this.maxDocsToCache = maxDocsToCache;
contexts = new ArrayList<>(); contexts = new ArrayList<>();
acceptDocsOutOfOrders = new ArrayList<>();
docs = new ArrayList<>(); docs = new ArrayList<>();
} }
@ -105,7 +103,6 @@ public abstract class CachingCollector extends FilterCollector {
final LeafCollector in = this.in.getLeafCollector(context); final LeafCollector in = this.in.getLeafCollector(context);
if (contexts != null) { if (contexts != null) {
contexts.add(context); contexts.add(context);
acceptDocsOutOfOrders.add(in.acceptsDocsOutOfOrder());
} }
if (maxDocsToCache >= 0) { if (maxDocsToCache >= 0) {
return lastCollector = wrap(in, maxDocsToCache); return lastCollector = wrap(in, maxDocsToCache);
@ -152,14 +149,7 @@ public abstract class CachingCollector extends FilterCollector {
assert docs.size() == contexts.size(); assert docs.size() == contexts.size();
for (int i = 0; i < contexts.size(); ++i) { for (int i = 0; i < contexts.size(); ++i) {
final LeafReaderContext context = contexts.get(i); final LeafReaderContext context = contexts.get(i);
final boolean docsInOrder = !acceptDocsOutOfOrders.get(i);
final LeafCollector collector = other.getLeafCollector(context); final LeafCollector collector = other.getLeafCollector(context);
if (!collector.acceptsDocsOutOfOrder() && !docsInOrder) {
throw new IllegalArgumentException(
"cannot replay: given collector does not support "
+ "out-of-order collection, while the wrapped collector does. "
+ "Therefore cached documents may be out-of-order.");
}
collect(collector, i); collect(collector, i);
} }
} }
@ -306,10 +296,6 @@ public abstract class CachingCollector extends FilterCollector {
*/ */
public static CachingCollector create(final boolean acceptDocsOutOfOrder, boolean cacheScores, double maxRAMMB) { public static CachingCollector create(final boolean acceptDocsOutOfOrder, boolean cacheScores, double maxRAMMB) {
Collector other = new SimpleCollector() { Collector other = new SimpleCollector() {
@Override
public boolean acceptsDocsOutOfOrder() {
return acceptDocsOutOfOrder;
}
@Override @Override
public void collect(int doc) {} public void collect(int doc) {}

View File

@ -134,14 +134,14 @@ public class ConstantScoreQuery extends Query {
} }
@Override @Override
public BulkScorer bulkScorer(LeafReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) throws IOException { public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
final DocIdSetIterator disi; final DocIdSetIterator disi;
if (filter != null) { if (filter != null) {
assert query == null; assert query == null;
return super.bulkScorer(context, scoreDocsInOrder, acceptDocs); return super.bulkScorer(context, acceptDocs);
} else { } else {
assert query != null && innerWeight != null; assert query != null && innerWeight != null;
BulkScorer bulkScorer = innerWeight.bulkScorer(context, scoreDocsInOrder, acceptDocs); BulkScorer bulkScorer = innerWeight.bulkScorer(context, acceptDocs);
if (bulkScorer == null) { if (bulkScorer == null) {
return null; return null;
} }
@ -170,11 +170,6 @@ public class ConstantScoreQuery extends Query {
return new ConstantScorer(disi, this, queryWeight); return new ConstantScorer(disi, this, queryWeight);
} }
@Override
public boolean scoresDocsOutOfOrder() {
return (innerWeight != null) ? innerWeight.scoresDocsOutOfOrder() : false;
}
@Override @Override
public Explanation explain(LeafReaderContext context, int doc) throws IOException { public Explanation explain(LeafReaderContext context, int doc) throws IOException {
final Scorer cs = scorer(context, context.reader().getLiveDocs()); final Scorer cs = scorer(context, context.reader().getLiveDocs());

View File

@ -43,11 +43,6 @@ public class FilterLeafCollector implements LeafCollector {
in.collect(doc); in.collect(doc);
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return in.acceptsDocsOutOfOrder();
}
@Override @Override
public String toString() { public String toString() {
return getClass().getSimpleName() + "(" + in + ")"; return getClass().getSimpleName() + "(" + in + ")";

View File

@ -82,11 +82,6 @@ public class FilteredQuery extends Query {
final Weight weight = query.createWeight (searcher); final Weight weight = query.createWeight (searcher);
return new Weight() { return new Weight() {
@Override
public boolean scoresDocsOutOfOrder() {
return true;
}
@Override @Override
public float getValueForNormalization() throws IOException { public float getValueForNormalization() throws IOException {
return weight.getValueForNormalization() * getBoost() * getBoost(); // boost sub-weight return weight.getValueForNormalization() * getBoost() * getBoost(); // boost sub-weight
@ -138,7 +133,7 @@ public class FilteredQuery extends Query {
// return a filtering top scorer // return a filtering top scorer
@Override @Override
public BulkScorer bulkScorer(LeafReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) throws IOException { public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
assert filter != null; assert filter != null;
DocIdSet filterDocIdSet = filter.getDocIdSet(context, acceptDocs); DocIdSet filterDocIdSet = filter.getDocIdSet(context, acceptDocs);
@ -147,7 +142,7 @@ public class FilteredQuery extends Query {
return null; return null;
} }
return strategy.filteredBulkScorer(context, weight, scoreDocsInOrder, filterDocIdSet); return strategy.filteredBulkScorer(context, weight, filterDocIdSet);
} }
}; };
} }
@ -480,7 +475,7 @@ public class FilteredQuery extends Query {
* @return a filtered top scorer * @return a filtered top scorer
*/ */
public BulkScorer filteredBulkScorer(LeafReaderContext context, public BulkScorer filteredBulkScorer(LeafReaderContext context,
Weight weight, boolean scoreDocsInOrder, DocIdSet docIdSet) throws IOException { Weight weight, DocIdSet docIdSet) throws IOException {
Scorer scorer = filteredScorer(context, weight, docIdSet); Scorer scorer = filteredScorer(context, weight, docIdSet);
if (scorer == null) { if (scorer == null) {
return null; return null;
@ -603,13 +598,12 @@ public class FilteredQuery extends Query {
@Override @Override
public BulkScorer filteredBulkScorer(final LeafReaderContext context, public BulkScorer filteredBulkScorer(final LeafReaderContext context,
Weight weight, Weight weight,
boolean scoreDocsInOrder, // ignored (we always top-score in order)
DocIdSet docIdSet) throws IOException { DocIdSet docIdSet) throws IOException {
Bits filterAcceptDocs = docIdSet.bits(); Bits filterAcceptDocs = docIdSet.bits();
if (filterAcceptDocs == null) { if (filterAcceptDocs == null) {
// Filter does not provide random-access Bits; we // Filter does not provide random-access Bits; we
// must fallback to leapfrog: // must fallback to leapfrog:
return LEAP_FROG_QUERY_FIRST_STRATEGY.filteredBulkScorer(context, weight, scoreDocsInOrder, docIdSet); return LEAP_FROG_QUERY_FIRST_STRATEGY.filteredBulkScorer(context, weight, docIdSet);
} }
final Scorer scorer = weight.scorer(context, null); final Scorer scorer = weight.scorer(context, null);
return scorer == null ? null : new QueryFirstBulkScorer(scorer, filterAcceptDocs); return scorer == null ? null : new QueryFirstBulkScorer(scorer, filterAcceptDocs);

View File

@ -479,7 +479,7 @@ public class IndexSearcher {
limit = 1; limit = 1;
} }
nDocs = Math.min(nDocs, limit); nDocs = Math.min(nDocs, limit);
TopScoreDocCollector collector = TopScoreDocCollector.create(nDocs, after, !weight.scoresDocsOutOfOrder()); TopScoreDocCollector collector = TopScoreDocCollector.create(nDocs, after);
search(leaves, weight, collector); search(leaves, weight, collector);
return collector.topDocs(); return collector.topDocs();
} }
@ -528,8 +528,7 @@ public class IndexSearcher {
after, after,
fillFields, fillFields,
doDocScores, doDocScores,
doMaxScore, doMaxScore);
false);
final Lock lock = new ReentrantLock(); final Lock lock = new ReentrantLock();
final ExecutionHelper<TopFieldDocs> runner = new ExecutionHelper<>(executor); final ExecutionHelper<TopFieldDocs> runner = new ExecutionHelper<>(executor);
@ -569,7 +568,7 @@ public class IndexSearcher {
TopFieldCollector collector = TopFieldCollector.create(sort, nDocs, after, TopFieldCollector collector = TopFieldCollector.create(sort, nDocs, after,
fillFields, doDocScores, fillFields, doDocScores,
doMaxScore, !weight.scoresDocsOutOfOrder()); doMaxScore);
search(leaves, weight, collector); search(leaves, weight, collector);
return (TopFieldDocs) collector.topDocs(); return (TopFieldDocs) collector.topDocs();
} }
@ -608,7 +607,7 @@ public class IndexSearcher {
// continue with the following leaf // continue with the following leaf
continue; continue;
} }
BulkScorer scorer = weight.bulkScorer(ctx, !leafCollector.acceptsDocsOutOfOrder(), ctx.reader().getLiveDocs()); BulkScorer scorer = weight.bulkScorer(ctx, ctx.reader().getLiveDocs());
if (scorer != null) { if (scorer != null) {
try { try {
scorer.score(leafCollector); scorer.score(leafCollector);

View File

@ -98,22 +98,4 @@ public interface LeafCollector {
*/ */
void collect(int doc) throws IOException; void collect(int doc) throws IOException;
/**
* Return <code>true</code> if this collector does not
* require the matching docIDs to be delivered in int sort
* order (smallest to largest) to {@link #collect}.
*
* <p> Most Lucene Query implementations will visit
* matching docIDs in order. However, some queries
* (currently limited to certain cases of {@link
* BooleanQuery}) can achieve faster searching if the
* <code>Collector</code> allows them to deliver the
* docIDs out of order.</p>
*
* <p> Many collectors don't mind getting docIDs out of
* order, so it's important to return <code>true</code>
* here.
*/
boolean acceptsDocsOutOfOrder();
} }

View File

@ -124,16 +124,6 @@ public class MultiCollector implements Collector {
} }
} }
@Override
public boolean acceptsDocsOutOfOrder() {
for (LeafCollector c : collectors) {
if (!c.acceptsDocsOutOfOrder()) {
return false;
}
}
return true;
}
} }
} }

View File

@ -44,9 +44,6 @@ public abstract class SimpleCollector implements Collector, LeafCollector {
// redeclare methods so that javadocs are inherited on sub-classes // redeclare methods so that javadocs are inherited on sub-classes
@Override
public abstract boolean acceptsDocsOutOfOrder();
@Override @Override
public abstract void collect(int doc) throws IOException; public abstract void collect(int doc) throws IOException;

View File

@ -53,7 +53,7 @@ public class SortRescorer extends Rescorer {
List<LeafReaderContext> leaves = searcher.getIndexReader().leaves(); List<LeafReaderContext> leaves = searcher.getIndexReader().leaves();
TopFieldCollector collector = TopFieldCollector.create(sort, topN, true, true, true, false); TopFieldCollector collector = TopFieldCollector.create(sort, topN, true, true, 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;

View File

@ -27,7 +27,7 @@ import org.apache.lucene.util.PriorityQueue;
* A {@link Collector} that sorts by {@link SortField} using * A {@link Collector} that sorts by {@link SortField} using
* {@link FieldComparator}s. * {@link FieldComparator}s.
* <p/> * <p/>
* See the {@link #create(org.apache.lucene.search.Sort, int, boolean, boolean, boolean, boolean)} method * See the {@link #create(org.apache.lucene.search.Sort, int, boolean, boolean, boolean)} method
* for instantiating a TopFieldCollector. * for instantiating a TopFieldCollector.
* *
* @lucene.experimental * @lucene.experimental
@ -55,11 +55,6 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
this.scorer = scorer; this.scorer = scorer;
comparator.setScorer(scorer); comparator.setScorer(scorer);
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
} }
private static abstract class MultiComparatorLeafCollector implements LeafCollector { private static abstract class MultiComparatorLeafCollector implements LeafCollector {
@ -124,11 +119,6 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
comparator.setScorer(scorer); comparator.setScorer(scorer);
} }
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
} }
/* /*
@ -218,102 +208,6 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
} }
/*
* Implements a TopFieldCollector over one SortField criteria, without
* tracking document scores and maxScore, and assumes out of orderness in doc
* Ids collection.
*/
private static class OutOfOrderNonScoringCollector extends TopFieldCollector {
final FieldValueHitQueue<Entry> queue;
public OutOfOrderNonScoringCollector(FieldValueHitQueue<Entry> queue, int numHits, boolean fillFields) {
super(queue, numHits, fillFields);
this.queue = queue;
}
@Override
public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
docBase = context.docBase;
final LeafFieldComparator[] comparators = queue.getComparators(context);
final int[] reverseMul = queue.getReverseMul();
if (comparators.length == 1) {
return new OneComparatorLeafCollector(comparators[0], reverseMul[0]) {
@Override
public void collect(int doc) throws IOException {
++totalHits;
if (queueFull) {
// Fastmatch: return if this hit is not competitive
final int cmp = reverseMul * comparator.compareBottom(doc);
if (cmp < 0 || (cmp == 0 && doc + docBase > bottom.doc)) {
return;
}
// This hit is competitive - replace bottom element in queue & adjustTop
comparator.copy(bottom.slot, doc);
updateBottom(doc);
comparator.setBottom(bottom.slot);
} else {
// Startup transient: queue hasn't gathered numHits yet
final int slot = totalHits - 1;
// Copy hit into queue
comparator.copy(slot, doc);
add(slot, doc, Float.NaN);
if (queueFull) {
comparator.setBottom(bottom.slot);
}
}
}
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
};
} else {
return new MultiComparatorLeafCollector(comparators, reverseMul) {
@Override
public void collect(int doc) throws IOException {
++totalHits;
if (queueFull) {
// Fastmatch: return if this hit is not competitive
final int cmp = compareBottom(doc);
if (cmp < 0 || (cmp == 0 && doc + docBase > bottom.doc)) {
return;
}
// This hit is competitive - replace bottom element in queue & adjustTop
copy(bottom.slot, doc);
updateBottom(doc);
setBottom(bottom.slot);
} else {
// Startup transient: queue hasn't gathered numHits yet
final int slot = totalHits - 1;
// Copy hit into queue
copy(slot, doc);
add(slot, doc, Float.NaN);
if (queueFull) {
setBottom(bottom.slot);
}
}
}
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
};
}
}
}
/* /*
* Implements a TopFieldCollector over one SortField criteria, while tracking * Implements a TopFieldCollector over one SortField criteria, while tracking
* document scores but no maxScore. * document scores but no maxScore.
@ -413,113 +307,6 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
} }
/*
* Implements a TopFieldCollector over one SortField criteria, while tracking
* document scores but no maxScore, and assumes out of orderness in doc Ids
* collection.
*/
private static class OutOfOrderScoringNoMaxScoreCollector extends TopFieldCollector {
final FieldValueHitQueue<Entry> queue;
public OutOfOrderScoringNoMaxScoreCollector(FieldValueHitQueue<Entry> queue, int numHits, boolean fillFields) {
super(queue, numHits, fillFields);
this.queue = queue;
}
@Override
public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
docBase = context.docBase;
final LeafFieldComparator[] comparators = queue.getComparators(context);
final int[] reverseMul = queue.getReverseMul();
if (comparators.length == 1) {
return new OneComparatorLeafCollector(comparators[0], reverseMul[0]) {
@Override
public void collect(int doc) throws IOException {
++totalHits;
if (queueFull) {
// Fastmatch: return if this hit is not competitive
final int cmp = reverseMul * comparator.compareBottom(doc);
if (cmp < 0 || (cmp == 0 && doc + docBase > bottom.doc)) {
return;
}
// Compute the score only if the hit is competitive.
final float score = scorer.score();
// This hit is competitive - replace bottom element in queue & adjustTop
comparator.copy(bottom.slot, doc);
updateBottom(doc, score);
comparator.setBottom(bottom.slot);
} else {
// Compute the score only if the hit is competitive.
final float score = scorer.score();
// Startup transient: queue hasn't gathered numHits yet
final int slot = totalHits - 1;
// Copy hit into queue
comparator.copy(slot, doc);
add(slot, doc, score);
if (queueFull) {
comparator.setBottom(bottom.slot);
}
}
}
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
};
} else {
return new MultiComparatorLeafCollector(comparators, reverseMul) {
@Override
public void collect(int doc) throws IOException {
++totalHits;
if (queueFull) {
// Fastmatch: return if this hit is not competitive
final int cmp = compareBottom(doc);
if (cmp < 0 || (cmp == 0 && doc + docBase > bottom.doc)) {
return;
}
// Compute the score only if the hit is competitive.
final float score = scorer.score();
// This hit is competitive - replace bottom element in queue & adjustTop
copy(bottom.slot, doc);
updateBottom(doc, score);
setBottom(bottom.slot);
} else {
// Compute the score only if the hit is competitive.
final float score = scorer.score();
// Startup transient: queue hasn't gathered numHits yet
final int slot = totalHits - 1;
// Copy hit into queue
copy(slot, doc);
add(slot, doc, score);
if (queueFull) {
setBottom(bottom.slot);
}
}
}
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
};
}
}
}
/* /*
* Implements a TopFieldCollector over one SortField criteria, with tracking * Implements a TopFieldCollector over one SortField criteria, with tracking
* document scores and maxScore. * document scores and maxScore.
@ -616,109 +403,6 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
} }
/*
* Implements a TopFieldCollector over one SortField criteria, with tracking
* document scores and maxScore, and assumes out of orderness in doc Ids
* collection.
*/
private static class OutOfOrderScoringMaxScoreCollector extends TopFieldCollector {
final FieldValueHitQueue<Entry> queue;
public OutOfOrderScoringMaxScoreCollector(FieldValueHitQueue<Entry> queue, int numHits, boolean fillFields) {
super(queue, numHits, fillFields);
this.queue = queue;
maxScore = Float.MIN_NORMAL; // otherwise we would keep NaN
}
@Override
public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
docBase = context.docBase;
final LeafFieldComparator[] comparators = queue.getComparators(context);
final int[] reverseMul = queue.getReverseMul();
if (comparators.length == 1) {
return new OneComparatorLeafCollector(comparators[0], reverseMul[0]) {
@Override
public void collect(int doc) throws IOException {
final float score = scorer.score();
if (score > maxScore) {
maxScore = score;
}
++totalHits;
if (queueFull) {
// Fastmatch: return if this hit is not competitive
final int cmp = reverseMul * comparator.compareBottom(doc);
if (cmp < 0 || (cmp == 0 && doc + docBase > bottom.doc)) {
return;
}
// This hit is competitive - replace bottom element in queue & adjustTop
comparator.copy(bottom.slot, doc);
updateBottom(doc, score);
comparator.setBottom(bottom.slot);
} else {
// Startup transient: queue hasn't gathered numHits yet
final int slot = totalHits - 1;
// Copy hit into queue
comparator.copy(slot, doc);
add(slot, doc, score);
if (queueFull) {
comparator.setBottom(bottom.slot);
}
}
}
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
};
} else {
return new MultiComparatorLeafCollector(comparators, reverseMul) {
@Override
public void collect(int doc) throws IOException {
final float score = scorer.score();
if (score > maxScore) {
maxScore = score;
}
++totalHits;
if (queueFull) {
// Fastmatch: return if this hit is not competitive
final int cmp = compareBottom(doc);
if (cmp < 0 || (cmp == 0 && doc + docBase > bottom.doc)) {
return;
}
// This hit is competitive - replace bottom element in queue & adjustTop
copy(bottom.slot, doc);
updateBottom(doc, score);
setBottom(bottom.slot);
} else {
// Startup transient: queue hasn't gathered numHits yet
final int slot = totalHits - 1;
// Copy hit into queue
copy(slot, doc);
add(slot, doc, score);
if (queueFull) {
setBottom(bottom.slot);
}
}
}
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
};
}
}
}
/* /*
* Implements a TopFieldCollector when after != null. * Implements a TopFieldCollector when after != null.
*/ */
@ -774,8 +458,8 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
// Fastmatch: return if this hit is no better than // Fastmatch: return if this hit is no better than
// the worst hit currently in the queue: // the worst hit currently in the queue:
final int cmp = compareBottom(doc); final int cmp = compareBottom(doc);
if (cmp < 0 || (cmp == 0 && doc + docBase > bottom.doc)) { if (cmp <= 0) {
// Definitely not competitive. // not competitive since documents are visited in doc id order
return; return;
} }
} }
@ -817,11 +501,6 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
} }
} }
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
}; };
} }
@ -882,18 +561,14 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
* true affects performance as it incurs the score computation on * true affects performance as it incurs the score computation on
* each result. Also, setting this true automatically sets * each result. Also, setting this true automatically sets
* <code>trackDocScores</code> to true as well. * <code>trackDocScores</code> to true as well.
* @param docsScoredInOrder
* specifies whether documents are scored in doc Id order or not by
* the given {@link Scorer} in {@link LeafCollector#setScorer(Scorer)}.
* @return a {@link TopFieldCollector} instance which will sort the results by * @return a {@link TopFieldCollector} instance which will sort the results by
* the sort criteria. * the sort criteria.
* @throws IOException if there is a low-level I/O error * @throws IOException if there is a low-level I/O error
*/ */
public static TopFieldCollector create(Sort sort, int numHits, public static TopFieldCollector create(Sort sort, int numHits,
boolean fillFields, boolean trackDocScores, boolean trackMaxScore, boolean fillFields, boolean trackDocScores, boolean trackMaxScore)
boolean docsScoredInOrder)
throws IOException { throws IOException {
return create(sort, numHits, null, fillFields, trackDocScores, trackMaxScore, docsScoredInOrder); return create(sort, numHits, null, fillFields, trackDocScores, trackMaxScore);
} }
/** /**
@ -927,16 +602,12 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
* true affects performance as it incurs the score computation on * true affects performance as it incurs the score computation on
* each result. Also, setting this true automatically sets * each result. Also, setting this true automatically sets
* <code>trackDocScores</code> to true as well. * <code>trackDocScores</code> to true as well.
* @param docsScoredInOrder
* specifies whether documents are scored in doc Id order or not by
* the given {@link Scorer} in {@link LeafCollector#setScorer(Scorer)}.
* @return a {@link TopFieldCollector} instance which will sort the results by * @return a {@link TopFieldCollector} instance which will sort the results by
* the sort criteria. * the sort criteria.
* @throws IOException if there is a low-level I/O error * @throws IOException if there is a low-level I/O error
*/ */
public static TopFieldCollector create(Sort sort, int numHits, FieldDoc after, public static TopFieldCollector create(Sort sort, int numHits, FieldDoc after,
boolean fillFields, boolean trackDocScores, boolean trackMaxScore, boolean fillFields, boolean trackDocScores, boolean trackMaxScore)
boolean docsScoredInOrder)
throws IOException { throws IOException {
if (sort.fields.length == 0) { if (sort.fields.length == 0) {
@ -950,7 +621,6 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
FieldValueHitQueue<Entry> queue = FieldValueHitQueue.create(sort.fields, numHits); FieldValueHitQueue<Entry> queue = FieldValueHitQueue.create(sort.fields, numHits);
if (after == null) { if (after == null) {
if (docsScoredInOrder) {
if (trackMaxScore) { if (trackMaxScore) {
return new ScoringMaxScoreCollector(queue, numHits, fillFields); return new ScoringMaxScoreCollector(queue, numHits, fillFields);
} else if (trackDocScores) { } else if (trackDocScores) {
@ -958,15 +628,6 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
} else { } else {
return new NonScoringCollector(queue, numHits, fillFields); return new NonScoringCollector(queue, numHits, fillFields);
} }
} else {
if (trackMaxScore) {
return new OutOfOrderScoringMaxScoreCollector(queue, numHits, fillFields);
} else if (trackDocScores) {
return new OutOfOrderScoringNoMaxScoreCollector(queue, numHits, fillFields);
} else {
return new OutOfOrderNonScoringCollector(queue, numHits, fillFields);
}
}
} else { } else {
if (after.fields == null) { if (after.fields == null) {
throw new IllegalArgumentException("after.fields wasn't set; you must pass fillFields=true for the previous search"); throw new IllegalArgumentException("after.fields wasn't set; you must pass fillFields=true for the previous search");

View File

@ -36,13 +36,7 @@ import org.apache.lucene.index.LeafReaderContext;
*/ */
public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> { public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
private abstract static class ScorerLeafCollector implements LeafCollector { abstract static class ScorerLeafCollector implements LeafCollector {
final boolean scoreDocsInOrder;
ScorerLeafCollector(boolean scoreDocsInOrder) {
this.scoreDocsInOrder = scoreDocsInOrder;
}
Scorer scorer; Scorer scorer;
@ -51,28 +45,19 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
this.scorer = scorer; this.scorer = scorer;
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return scoreDocsInOrder == false;
}
} }
private static class SimpleTopScoreDocCollector extends TopScoreDocCollector { private static class SimpleTopScoreDocCollector extends TopScoreDocCollector {
private final boolean scoreDocsInOrder; SimpleTopScoreDocCollector(int numHits) {
SimpleTopScoreDocCollector(int numHits, boolean scoreDocsInOrder) {
super(numHits); super(numHits);
this.scoreDocsInOrder = scoreDocsInOrder;
} }
@Override @Override
public LeafCollector getLeafCollector(LeafReaderContext context) public LeafCollector getLeafCollector(LeafReaderContext context)
throws IOException { throws IOException {
final int docBase = context.docBase; final int docBase = context.docBase;
if (scoreDocsInOrder) { return new ScorerLeafCollector() {
return new ScorerLeafCollector(scoreDocsInOrder) {
@Override @Override
public void collect(int doc) throws IOException { public void collect(int doc) throws IOException {
@ -95,46 +80,17 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
} }
}; };
} else {
return new ScorerLeafCollector(scoreDocsInOrder) {
@Override
public void collect(int doc) throws IOException {
float score = scorer.score();
// This collector cannot handle NaN
assert !Float.isNaN(score);
totalHits++;
if (score < pqTop.score) {
// Doesn't compete w/ bottom entry in queue
return;
}
doc += docBase;
if (score == pqTop.score && doc > pqTop.doc) {
// Break tie in score by doc ID:
return;
}
pqTop.doc = doc;
pqTop.score = score;
pqTop = pq.updateTop();
}
};
}
} }
} }
private static class PagingTopScoreDocCollector extends TopScoreDocCollector { private static class PagingTopScoreDocCollector extends TopScoreDocCollector {
private final boolean scoreDocsInOrder;
private final ScoreDoc after; private final ScoreDoc after;
private int collectedHits; private int collectedHits;
PagingTopScoreDocCollector(int numHits, boolean scoreDocsInOrder, ScoreDoc after) { PagingTopScoreDocCollector(int numHits, ScoreDoc after) {
super(numHits); super(numHits);
this.scoreDocsInOrder = scoreDocsInOrder;
this.after = after; this.after = after;
this.collectedHits = 0; this.collectedHits = 0;
} }
@ -153,8 +109,7 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException { public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
final int docBase = context.docBase; final int docBase = context.docBase;
final int afterDoc = after.doc - context.docBase; final int afterDoc = after.doc - context.docBase;
if (scoreDocsInOrder) { return new ScorerLeafCollector() {
return new ScorerLeafCollector(scoreDocsInOrder) {
@Override @Override
public void collect(int doc) throws IOException { public void collect(int doc) throws IOException {
float score = scorer.score(); float score = scorer.score();
@ -182,36 +137,6 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
pqTop = pq.updateTop(); pqTop = pq.updateTop();
} }
}; };
} else {
return new ScorerLeafCollector(scoreDocsInOrder) {
@Override
public void collect(int doc) throws IOException {
float score = scorer.score();
// This collector cannot handle NaN
assert !Float.isNaN(score);
totalHits++;
if (score > after.score || (score == after.score && doc <= afterDoc)) {
// hit was collected on a previous page
return;
}
if (score < pqTop.score) {
// Doesn't compete w/ bottom entry in queue
return;
}
doc += docBase;
if (score == pqTop.score && doc > pqTop.doc) {
// Break tie in score by doc ID:
return;
}
collectedHits++;
pqTop.doc = doc;
pqTop.score = score;
pqTop = pq.updateTop();
}
};
}
} }
} }
@ -225,8 +150,8 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
* <code>numHits</code>, and fill the array with sentinel * <code>numHits</code>, and fill the array with sentinel
* objects. * objects.
*/ */
public static TopScoreDocCollector create(int numHits, boolean docsScoredInOrder) { public static TopScoreDocCollector create(int numHits) {
return create(numHits, null, docsScoredInOrder); return create(numHits, null);
} }
/** /**
@ -239,23 +164,23 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
* <code>numHits</code>, and fill the array with sentinel * <code>numHits</code>, and fill the array with sentinel
* objects. * objects.
*/ */
public static TopScoreDocCollector create(int numHits, ScoreDoc after, boolean docsScoredInOrder) { public static TopScoreDocCollector create(int numHits, ScoreDoc after) {
if (numHits <= 0) { if (numHits <= 0) {
throw new IllegalArgumentException("numHits must be > 0; please use TotalHitCountCollector if you just need the total hit count"); throw new IllegalArgumentException("numHits must be > 0; please use TotalHitCountCollector if you just need the total hit count");
} }
if (after == null) { if (after == null) {
return new SimpleTopScoreDocCollector(numHits, docsScoredInOrder); return new SimpleTopScoreDocCollector(numHits);
} else { } else {
return new PagingTopScoreDocCollector(numHits, docsScoredInOrder, after); return new PagingTopScoreDocCollector(numHits, after);
} }
} }
ScoreDoc pqTop; ScoreDoc pqTop;
// prevents instantiation // prevents instantiation
private TopScoreDocCollector(int numHits) { TopScoreDocCollector(int numHits) {
super(new HitQueue(numHits, true)); super(new HitQueue(numHits, true));
// HitQueue implements getSentinelObject to return a ScoreDoc, so we know // HitQueue implements getSentinelObject to return a ScoreDoc, so we know
// that at this point top() is already initialized. // that at this point top() is already initialized.

View File

@ -34,9 +34,4 @@ public class TotalHitCountCollector extends SimpleCollector {
public void collect(int doc) { public void collect(int doc) {
totalHits++; totalHits++;
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
} }

View File

@ -79,11 +79,6 @@ public abstract class Weight {
* Returns a {@link Scorer} which scores documents in/out-of order according * Returns a {@link Scorer} which scores documents in/out-of order according
* to <code>scoreDocsInOrder</code>. * to <code>scoreDocsInOrder</code>.
* <p> * <p>
* <b>NOTE:</b> even if <code>scoreDocsInOrder</code> is false, it is
* recommended to check whether the returned <code>Scorer</code> indeed scores
* documents out of order (i.e., call {@link #scoresDocsOutOfOrder()}), as
* some <code>Scorer</code> implementations will always return documents
* in-order.<br>
* <b>NOTE:</b> null can be returned if no documents will be scored by this * <b>NOTE:</b> null can be returned if no documents will be scored by this
* query. * query.
* *
@ -108,15 +103,6 @@ public abstract class Weight {
* *
* @param context * @param context
* the {@link org.apache.lucene.index.LeafReaderContext} for which to return the {@link Scorer}. * the {@link org.apache.lucene.index.LeafReaderContext} for which to return the {@link Scorer}.
* @param scoreDocsInOrder
* specifies whether in-order scoring of documents is required. Note
* that if set to false (i.e., out-of-order scoring is required),
* this method can return whatever scoring mode it supports, as every
* in-order scorer is also an out-of-order one. However, an
* out-of-order scorer may not support {@link Scorer#nextDoc()}
* and/or {@link Scorer#advance(int)}, therefore it is recommended to
* request an in-order scorer if use of these
* methods is required.
* @param acceptDocs * @param acceptDocs
* Bits that represent the allowable docs to match (typically deleted docs * Bits that represent the allowable docs to match (typically deleted docs
* but possibly filtering other documents) * but possibly filtering other documents)
@ -125,7 +111,7 @@ public abstract class Weight {
* passes them to a collector. * passes them to a collector.
* @throws IOException if there is a low-level I/O error * @throws IOException if there is a low-level I/O error
*/ */
public BulkScorer bulkScorer(LeafReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) throws IOException { public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
Scorer scorer = scorer(context, acceptDocs); Scorer scorer = scorer(context, acceptDocs);
if (scorer == null) { if (scorer == null) {
@ -193,19 +179,4 @@ public abstract class Weight {
} }
} }
} }
/**
* Returns true iff this implementation scores docs only out of order. This
* method is used in conjunction with {@link Collector}'s
* {@link LeafCollector#acceptsDocsOutOfOrder() acceptsDocsOutOfOrder} and
* {@link #bulkScorer(org.apache.lucene.index.LeafReaderContext, boolean, Bits)} to
* create a matching {@link Scorer} instance for a given {@link Collector}, or
* vice versa.
* <p>
* <b>NOTE:</b> the default implementation returns <code>false</code>, i.e.
* the <code>Scorer</code> scores documents in-order.
*/
public boolean scoresDocsOutOfOrder() {
return false;
}
} }

View File

@ -443,8 +443,8 @@ on the built-in available scoring models and extending or changing Similarity.
given the Query. given the Query.
</li> </li>
<li> <li>
{@link org.apache.lucene.search.Weight#bulkScorer(org.apache.lucene.index.LeafReaderContext, boolean, org.apache.lucene.util.Bits) {@link org.apache.lucene.search.Weight#bulkScorer(org.apache.lucene.index.LeafReaderContext, org.apache.lucene.util.Bits)
scorer(LeafReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs)} &mdash; scorer(LeafReaderContext context, Bits acceptDocs)} &mdash;
Construct a new {@link org.apache.lucene.search.BulkScorer BulkScorer} for this Weight. See <a href="#bulkScorerClass">The BulkScorer Class</a> Construct a new {@link org.apache.lucene.search.BulkScorer BulkScorer} for this Weight. See <a href="#bulkScorerClass">The BulkScorer Class</a>
below for help defining a BulkScorer. This is an optional method, and most queries do not implement it. below for help defining a BulkScorer. This is an optional method, and most queries do not implement it.
</li> </li>

View File

@ -437,10 +437,6 @@ public class TestOmitTf extends LuceneTestCase {
protected void doSetNextReader(LeafReaderContext context) throws IOException { protected void doSetNextReader(LeafReaderContext context) throws IOException {
docBase = context.docBase; docBase = context.docBase;
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
} }
/** test that when freqs are omitted, that totalTermFreq and sumTotalTermFreq are -1 */ /** test that when freqs are omitted, that totalTermFreq and sumTotalTermFreq are -1 */

View File

@ -53,11 +53,6 @@ final class JustCompileSearch {
throw new UnsupportedOperationException(UNSUPPORTED_MSG); throw new UnsupportedOperationException(UNSUPPORTED_MSG);
} }
@Override
public boolean acceptsDocsOutOfOrder() {
throw new UnsupportedOperationException(UNSUPPORTED_MSG);
}
} }
static final class JustCompileDocIdSet extends DocIdSet { static final class JustCompileDocIdSet extends DocIdSet {

View File

@ -27,17 +27,10 @@ public class MultiCollectorTest extends LuceneTestCase {
private static class DummyCollector extends SimpleCollector { private static class DummyCollector extends SimpleCollector {
boolean acceptsDocsOutOfOrderCalled = false;
boolean collectCalled = false; boolean collectCalled = false;
boolean setNextReaderCalled = false; boolean setNextReaderCalled = false;
boolean setScorerCalled = false; boolean setScorerCalled = false;
@Override
public boolean acceptsDocsOutOfOrder() {
acceptsDocsOutOfOrderCalled = true;
return true;
}
@Override @Override
public void collect(int doc) throws IOException { public void collect(int doc) throws IOException {
collectCalled = true; collectCalled = true;
@ -70,7 +63,6 @@ public class MultiCollectorTest extends LuceneTestCase {
Collector c = MultiCollector.wrap(new DummyCollector(), null, new DummyCollector()); Collector c = MultiCollector.wrap(new DummyCollector(), null, new DummyCollector());
assertTrue(c instanceof MultiCollector); assertTrue(c instanceof MultiCollector);
final LeafCollector ac = c.getLeafCollector(null); final LeafCollector ac = c.getLeafCollector(null);
assertTrue(ac.acceptsDocsOutOfOrder());
ac.collect(1); ac.collect(1);
c.getLeafCollector(null); c.getLeafCollector(null);
c.getLeafCollector(null).setScorer(null); c.getLeafCollector(null).setScorer(null);
@ -93,13 +85,11 @@ public class MultiCollectorTest extends LuceneTestCase {
DummyCollector[] dcs = new DummyCollector[] { new DummyCollector(), new DummyCollector() }; DummyCollector[] dcs = new DummyCollector[] { new DummyCollector(), new DummyCollector() };
Collector c = MultiCollector.wrap(dcs); Collector c = MultiCollector.wrap(dcs);
LeafCollector ac = c.getLeafCollector(null); LeafCollector ac = c.getLeafCollector(null);
assertTrue(ac.acceptsDocsOutOfOrder());
ac.collect(1); ac.collect(1);
ac = c.getLeafCollector(null); ac = c.getLeafCollector(null);
ac.setScorer(null); ac.setScorer(null);
for (DummyCollector dc : dcs) { for (DummyCollector dc : dcs) {
assertTrue(dc.acceptsDocsOutOfOrderCalled);
assertTrue(dc.collectCalled); assertTrue(dc.collectCalled);
assertTrue(dc.setNextReaderCalled); assertTrue(dc.setNextReaderCalled);
assertTrue(dc.setScorerCalled); assertTrue(dc.setScorerCalled);

View File

@ -130,11 +130,14 @@ public class TestBoolean2 extends LuceneTestCase {
}; };
public void queriesTest(Query query, int[] expDocNrs) throws Exception { public void queriesTest(Query query, int[] expDocNrs) throws Exception {
TopScoreDocCollector collector = TopScoreDocCollector.create(1000, false); // The asserting searcher will sometimes return the bulk scorer and
// sometimes return a default impl around the scorer so that we can
// compare BS1 and BS2
TopScoreDocCollector collector = TopScoreDocCollector.create(1000);
searcher.search(query, null, collector); searcher.search(query, null, collector);
ScoreDoc[] hits1 = collector.topDocs().scoreDocs; ScoreDoc[] hits1 = collector.topDocs().scoreDocs;
collector = TopScoreDocCollector.create(1000, true); collector = TopScoreDocCollector.create(1000);
searcher.search(query, null, collector); searcher.search(query, null, collector);
ScoreDoc[] hits2 = collector.topDocs().scoreDocs; ScoreDoc[] hits2 = collector.topDocs().scoreDocs;
@ -282,13 +285,13 @@ public class TestBoolean2 extends LuceneTestCase {
} }
TopFieldCollector collector = TopFieldCollector.create(sort, 1000, TopFieldCollector collector = TopFieldCollector.create(sort, 1000,
false, true, true, true); false, true, true);
searcher.search(q1, null, collector); searcher.search(q1, null, collector);
ScoreDoc[] hits1 = collector.topDocs().scoreDocs; ScoreDoc[] hits1 = collector.topDocs().scoreDocs;
collector = TopFieldCollector.create(sort, 1000, collector = TopFieldCollector.create(sort, 1000,
false, true, true, false); false, true, true);
searcher.search(q1, null, collector); searcher.search(q1, null, collector);
ScoreDoc[] hits2 = collector.topDocs().scoreDocs; ScoreDoc[] hits2 = collector.topDocs().scoreDocs;

View File

@ -712,10 +712,9 @@ public class TestBooleanCoord extends LuceneTestCase {
assertEquals(0, scorer.nextDoc()); assertEquals(0, scorer.nextDoc());
assertEquals(expected, scorer.score(), 0.0001f); assertEquals(expected, scorer.score(), 0.0001f);
// test out-of-order (if supported) // test bulk scorer
if (weight.scoresDocsOutOfOrder()) {
final AtomicBoolean seen = new AtomicBoolean(false); final AtomicBoolean seen = new AtomicBoolean(false);
BulkScorer bulkScorer = weight.bulkScorer(reader.leaves().get(0), false, null); BulkScorer bulkScorer = weight.bulkScorer(reader.leaves().get(0), null);
assertNotNull(bulkScorer); assertNotNull(bulkScorer);
bulkScorer.score(new LeafCollector() { bulkScorer.score(new LeafCollector() {
Scorer scorer; Scorer scorer;
@ -732,14 +731,8 @@ public class TestBooleanCoord extends LuceneTestCase {
assertEquals(expected, scorer.score(), 0.0001f); assertEquals(expected, scorer.score(), 0.0001f);
seen.set(true); seen.set(true);
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
}, 1); }, 1);
assertTrue(seen.get()); assertTrue(seen.get());
}
// test the explanation // test the explanation
Explanation expl = weight.explain(reader.leaves().get(0), 0); Explanation expl = weight.explain(reader.leaves().get(0), 0);

View File

@ -94,7 +94,7 @@ public class TestBooleanMinShouldMatch extends LuceneTestCase {
assertEquals("result count", expected, h.length); assertEquals("result count", expected, h.length);
//System.out.println("TEST: now check"); //System.out.println("TEST: now check");
// bs2 // bs2
TopScoreDocCollector collector = TopScoreDocCollector.create(1000, true); TopScoreDocCollector collector = TopScoreDocCollector.create(1000);
s.search(q, collector); s.search(q, collector);
ScoreDoc[] h2 = collector.topDocs().scoreDocs; ScoreDoc[] h2 = collector.topDocs().scoreDocs;
if (expected != h2.length) { if (expected != h2.length) {

View File

@ -182,7 +182,7 @@ public class TestBooleanOr extends LuceneTestCase {
Weight w = s.createNormalizedWeight(bq); Weight w = s.createNormalizedWeight(bq);
assertEquals(1, s.getIndexReader().leaves().size()); assertEquals(1, s.getIndexReader().leaves().size());
BulkScorer scorer = w.bulkScorer(s.getIndexReader().leaves().get(0), false, null); BulkScorer scorer = w.bulkScorer(s.getIndexReader().leaves().get(0), null);
final FixedBitSet hits = new FixedBitSet(docCount); final FixedBitSet hits = new FixedBitSet(docCount);
final AtomicInteger end = new AtomicInteger(); final AtomicInteger end = new AtomicInteger();
@ -193,11 +193,6 @@ public class TestBooleanOr extends LuceneTestCase {
assertTrue("collected doc=" + doc + " beyond max=" + end, doc < end.intValue()); assertTrue("collected doc=" + doc + " beyond max=" + end, doc < end.intValue());
hits.set(doc); hits.set(doc);
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
}; };
while (end.intValue() < docCount) { while (end.intValue() < docCount) {

View File

@ -314,7 +314,7 @@ public class TestBooleanQuery extends LuceneTestCase {
SpanQuery sq2 = new SpanTermQuery(new Term(FIELD, "clckwork")); SpanQuery sq2 = new SpanTermQuery(new Term(FIELD, "clckwork"));
query.add(sq1, BooleanClause.Occur.SHOULD); query.add(sq1, BooleanClause.Occur.SHOULD);
query.add(sq2, BooleanClause.Occur.SHOULD); query.add(sq2, BooleanClause.Occur.SHOULD);
TopScoreDocCollector collector = TopScoreDocCollector.create(1000, true); TopScoreDocCollector collector = TopScoreDocCollector.create(1000);
searcher.search(query, collector); searcher.search(query, collector);
hits = collector.topDocs().scoreDocs.length; hits = collector.topDocs().scoreDocs.length;
for (ScoreDoc scoreDoc : collector.topDocs().scoreDocs){ for (ScoreDoc scoreDoc : collector.topDocs().scoreDocs){

View File

@ -36,13 +36,17 @@ import org.apache.lucene.index.IndexWriterConfig;
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.BooleanClause.Occur; import org.apache.lucene.search.BooleanClause.Occur;
import org.apache.lucene.search.BooleanQuery.BooleanWeight;
import org.apache.lucene.search.Scorer.ChildScorer; import org.apache.lucene.search.Scorer.ChildScorer;
import org.apache.lucene.search.Weight.DefaultBulkScorer;
import org.apache.lucene.store.Directory; import org.apache.lucene.store.Directory;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.LuceneTestCase;
// TODO: refactor to a base class, that collects freqs from the scorer tree // TODO: refactor to a base class, that collects freqs from the scorer tree
// and test all queries with it // and test all queries with it
public class TestBooleanQueryVisitSubscorers extends LuceneTestCase { public class TestBooleanQueryVisitSubscorers extends LuceneTestCase {
Analyzer analyzer; Analyzer analyzer;
IndexReader reader; IndexReader reader;
IndexSearcher searcher; IndexSearcher searcher;
@ -64,7 +68,9 @@ public class TestBooleanQueryVisitSubscorers extends LuceneTestCase {
writer.addDocument(doc("nutch", "nutch is an internet search engine with web crawler and is using lucene and hadoop")); writer.addDocument(doc("nutch", "nutch is an internet search engine with web crawler and is using lucene and hadoop"));
reader = writer.getReader(); reader = writer.getReader();
writer.close(); writer.close();
searcher = newSearcher(reader); // we do not use newSearcher because the assertingXXX layers break
// the toString representations we are relying on
searcher = new IndexSearcher(reader);
} }
@Override @Override
@ -75,7 +81,7 @@ public class TestBooleanQueryVisitSubscorers extends LuceneTestCase {
} }
public void testDisjunctions() throws IOException { public void testDisjunctions() throws IOException {
BooleanQuery bq = new BooleanQuery(); BooleanQuery2 bq = new BooleanQuery2();
bq.add(new TermQuery(new Term(F1, "lucene")), BooleanClause.Occur.SHOULD); bq.add(new TermQuery(new Term(F1, "lucene")), BooleanClause.Occur.SHOULD);
bq.add(new TermQuery(new Term(F2, "lucene")), BooleanClause.Occur.SHOULD); bq.add(new TermQuery(new Term(F2, "lucene")), BooleanClause.Occur.SHOULD);
bq.add(new TermQuery(new Term(F2, "search")), BooleanClause.Occur.SHOULD); bq.add(new TermQuery(new Term(F2, "search")), BooleanClause.Occur.SHOULD);
@ -87,9 +93,9 @@ public class TestBooleanQueryVisitSubscorers extends LuceneTestCase {
} }
public void testNestedDisjunctions() throws IOException { public void testNestedDisjunctions() throws IOException {
BooleanQuery bq = new BooleanQuery(); BooleanQuery2 bq = new BooleanQuery2();
bq.add(new TermQuery(new Term(F1, "lucene")), BooleanClause.Occur.SHOULD); bq.add(new TermQuery(new Term(F1, "lucene")), BooleanClause.Occur.SHOULD);
BooleanQuery bq2 = new BooleanQuery(); BooleanQuery2 bq2 = new BooleanQuery2();
bq2.add(new TermQuery(new Term(F2, "lucene")), BooleanClause.Occur.SHOULD); bq2.add(new TermQuery(new Term(F2, "lucene")), BooleanClause.Occur.SHOULD);
bq2.add(new TermQuery(new Term(F2, "search")), BooleanClause.Occur.SHOULD); bq2.add(new TermQuery(new Term(F2, "search")), BooleanClause.Occur.SHOULD);
bq.add(bq2, BooleanClause.Occur.SHOULD); bq.add(bq2, BooleanClause.Occur.SHOULD);
@ -130,7 +136,7 @@ public class TestBooleanQueryVisitSubscorers extends LuceneTestCase {
private final Set<Scorer> tqsSet = new HashSet<>(); private final Set<Scorer> tqsSet = new HashSet<>();
MyCollector() { MyCollector() {
super(TopScoreDocCollector.create(10, true)); super(TopScoreDocCollector.create(10));
} }
public LeafCollector getLeafCollector(LeafReaderContext context) public LeafCollector getLeafCollector(LeafReaderContext context)
@ -138,11 +144,6 @@ public class TestBooleanQueryVisitSubscorers extends LuceneTestCase {
final int docBase = context.docBase; final int docBase = context.docBase;
return new FilterLeafCollector(super.getLeafCollector(context)) { return new FilterLeafCollector(super.getLeafCollector(context)) {
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
@Override @Override
public void setScorer(Scorer scorer) throws IOException { public void setScorer(Scorer scorer) throws IOException {
super.setScorer(scorer); super.setScorer(scorer);
@ -205,7 +206,7 @@ public class TestBooleanQueryVisitSubscorers extends LuceneTestCase {
} }
public void testGetChildrenBoosterScorer() throws IOException { public void testGetChildrenBoosterScorer() throws IOException {
final BooleanQuery query = new BooleanQuery(); final BooleanQuery2 query = new BooleanQuery2();
query.add(new TermQuery(new Term(F2, "nutch")), Occur.SHOULD); query.add(new TermQuery(new Term(F2, "nutch")), Occur.SHOULD);
query.add(new TermQuery(new Term(F2, "miss")), Occur.SHOULD); query.add(new TermQuery(new Term(F2, "miss")), Occur.SHOULD);
ScorerSummarizingCollector collector = new ScorerSummarizingCollector(); ScorerSummarizingCollector collector = new ScorerSummarizingCollector();
@ -246,11 +247,6 @@ public class TestBooleanQueryVisitSubscorers extends LuceneTestCase {
public void collect(int doc) throws IOException { public void collect(int doc) throws IOException {
numHits[0]++; numHits[0]++;
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
}; };
} }
@ -276,4 +272,22 @@ public class TestBooleanQueryVisitSubscorers extends LuceneTestCase {
return builder; return builder;
} }
} }
static class BooleanQuery2 extends BooleanQuery {
@Override
public Weight createWeight(IndexSearcher searcher) throws IOException {
return new BooleanWeight(searcher, false) {
@Override
public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
Scorer scorer = scorer(context, acceptDocs);
if (scorer == null) {
return null;
}
return new DefaultBulkScorer(scorer);
}
};
}
}
} }

View File

@ -66,66 +66,6 @@ public class TestBooleanScorer extends LuceneTestCase {
directory.close(); directory.close();
} }
public void testEmptyBucketWithMoreDocs() throws Exception {
// This test checks the logic of nextDoc() when all sub scorers have docs
// beyond the first bucket (for example). Currently, the code relies on the
// 'more' variable to work properly, and this test ensures that if the logic
// changes, we have a test to back it up.
Directory directory = newDirectory();
RandomIndexWriter writer = new RandomIndexWriter(random(), directory);
writer.commit();
IndexReader ir = writer.getReader();
writer.close();
IndexSearcher searcher = newSearcher(ir);
BooleanWeight weight = (BooleanWeight) new BooleanQuery().createWeight(searcher);
BulkScorer[] scorers = new BulkScorer[] {new BulkScorer() {
private int doc = -1;
@Override
public boolean score(LeafCollector c, int maxDoc) throws IOException {
assert doc == -1;
doc = 3000;
FakeScorer fs = new FakeScorer();
fs.doc = doc;
fs.score = 1.0f;
c.setScorer(fs);
c.collect(3000);
return false;
}
}};
BooleanScorer bs = new BooleanScorer(weight, false, 1, Arrays.asList(scorers), Collections.<BulkScorer>emptyList(), scorers.length);
final List<Integer> hits = new ArrayList<>();
bs.score(new SimpleCollector() {
int docBase;
@Override
public void setScorer(Scorer scorer) {
}
@Override
public void collect(int doc) {
hits.add(docBase+doc);
}
@Override
protected void doSetNextReader(LeafReaderContext context) throws IOException {
docBase = context.docBase;
}
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
});
assertEquals("should have only 1 hit", 1, hits.size());
assertEquals("hit should have been docID=3000", 3000, hits.get(0).intValue());
ir.close();
directory.close();
}
/** Throws UOE if Weight.scorer is called */ /** Throws UOE if Weight.scorer is called */
private static class CrazyMustUseBulkScorerQuery extends Query { private static class CrazyMustUseBulkScorerQuery extends Query {
@ -162,7 +102,7 @@ public class TestBooleanScorer extends LuceneTestCase {
} }
@Override @Override
public BulkScorer bulkScorer(LeafReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) { public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs) {
return new BulkScorer() { return new BulkScorer() {
@Override @Override

View File

@ -1,133 +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.Field;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.Term;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.LuceneTestCase;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
/**
* BooleanQuery.scorer should be tested, when hit documents
* are very unevenly distributed.
*/
public class TestBooleanUnevenly extends LuceneTestCase {
private static IndexSearcher searcher;
private static IndexReader reader;
public static final String field = "field";
private static Directory directory;
private static int count1;
@BeforeClass
public static void beforeClass() throws Exception {
directory = newDirectory();
RandomIndexWriter w = new RandomIndexWriter(random(), directory, new MockAnalyzer(random()));
Document doc;
count1 = 0;
for (int i=0;i<2;i++) {
for (int j=0;j<2048;j++) {
doc = new Document();
doc.add(newTextField(field, "1", Field.Store.NO));
count1 ++;
w.addDocument(doc);
}
for (int j=0;j<2048;j++) {
doc = new Document();
doc.add(newTextField(field, "2", Field.Store.NO));
w.addDocument(doc);
}
doc = new Document();
doc.add(newTextField(field, "1", Field.Store.NO));
count1 ++;
w.addDocument(doc);
for (int j=0;j<2048;j++) {
doc = new Document();
doc.add(newTextField(field, "2", Field.Store.NO));
w.addDocument(doc);
}
}
reader = w.getReader();
searcher = newSearcher(reader);
w.close();
}
@AfterClass
public static void afterClass() throws Exception {
reader.close();
directory.close();
searcher = null;
reader = null;
directory = null;
}
@Test
public void testQueries01() throws Exception {
BooleanQuery query = new BooleanQuery();
query.add(new TermQuery(new Term(field, "1")), BooleanClause.Occur.MUST);
query.add(new TermQuery(new Term(field, "1")), BooleanClause.Occur.SHOULD);
query.add(new TermQuery(new Term(field, "2")), BooleanClause.Occur.SHOULD);
TopScoreDocCollector collector = TopScoreDocCollector.create(1000, false);
searcher.search(query, null, collector);
TopDocs tops1 = collector.topDocs();
ScoreDoc[] hits1 = tops1.scoreDocs;
int hitsNum1 = tops1.totalHits;
collector = TopScoreDocCollector.create(1000, true);
searcher.search(query, null, collector);
TopDocs tops2 = collector.topDocs();
ScoreDoc[] hits2 = tops2.scoreDocs;
int hitsNum2 = tops2.totalHits;
assertEquals(count1, hitsNum1);
assertEquals(count1, hitsNum2);
CheckHits.checkEqual(query, hits1, hits2);
}
@Test
public void testQueries02() throws Exception {
BooleanQuery query = new BooleanQuery();
query.add(new TermQuery(new Term(field, "1")), BooleanClause.Occur.SHOULD);
query.add(new TermQuery(new Term(field, "1")), BooleanClause.Occur.SHOULD);
TopScoreDocCollector collector = TopScoreDocCollector.create(1000, false);
searcher.search(query, null, collector);
TopDocs tops1 = collector.topDocs();
ScoreDoc[] hits1 = tops1.scoreDocs;
int hitsNum1 = tops1.totalHits;
collector = TopScoreDocCollector.create(1000, true);
searcher.search(query, null, collector);
TopDocs tops2 = collector.topDocs();
ScoreDoc[] hits2 = tops2.scoreDocs;
int hitsNum2 = tops2.totalHits;
assertEquals(count1, hitsNum1);
assertEquals(count1, hitsNum2);
CheckHits.checkEqual(query, hits1, hits2);
}
}

View File

@ -54,25 +54,14 @@ public class TestCachingCollector extends LuceneTestCase {
private static class NoOpCollector extends SimpleCollector { private static class NoOpCollector extends SimpleCollector {
private final boolean acceptDocsOutOfOrder;
public NoOpCollector(boolean acceptDocsOutOfOrder) {
this.acceptDocsOutOfOrder = acceptDocsOutOfOrder;
}
@Override @Override
public void collect(int doc) throws IOException {} public void collect(int doc) throws IOException {}
@Override
public boolean acceptsDocsOutOfOrder() {
return acceptDocsOutOfOrder;
}
} }
public void testBasic() throws Exception { public void testBasic() throws Exception {
for (boolean cacheScores : new boolean[] { false, true }) { for (boolean cacheScores : new boolean[] { false, true }) {
CachingCollector cc = CachingCollector.create(new NoOpCollector(false), cacheScores, 1.0); CachingCollector cc = CachingCollector.create(new NoOpCollector(), cacheScores, 1.0);
LeafCollector acc = cc.getLeafCollector(null); LeafCollector acc = cc.getLeafCollector(null);
acc.setScorer(new MockScorer()); acc.setScorer(new MockScorer());
@ -90,17 +79,12 @@ public class TestCachingCollector extends LuceneTestCase {
assertEquals(prevDocID + 1, doc); assertEquals(prevDocID + 1, doc);
prevDocID = doc; prevDocID = doc;
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
}); });
} }
} }
public void testIllegalStateOnReplay() throws Exception { public void testIllegalStateOnReplay() throws Exception {
CachingCollector cc = CachingCollector.create(new NoOpCollector(false), true, 50 * ONE_BYTE); CachingCollector cc = CachingCollector.create(new NoOpCollector(), true, 50 * ONE_BYTE);
LeafCollector acc = cc.getLeafCollector(null); LeafCollector acc = cc.getLeafCollector(null);
acc.setScorer(new MockScorer()); acc.setScorer(new MockScorer());
@ -112,40 +96,13 @@ public class TestCachingCollector extends LuceneTestCase {
assertFalse("CachingCollector should not be cached due to low memory limit", cc.isCached()); assertFalse("CachingCollector should not be cached due to low memory limit", cc.isCached());
try { try {
cc.replay(new NoOpCollector(false)); cc.replay(new NoOpCollector());
fail("replay should fail if CachingCollector is not cached"); fail("replay should fail if CachingCollector is not cached");
} catch (IllegalStateException e) { } catch (IllegalStateException e) {
// expected // expected
} }
} }
public void testIllegalCollectorOnReplay() throws Exception {
// tests that the Collector passed to replay() has an out-of-order mode that
// is valid with the Collector passed to the ctor
// 'src' Collector does not support out-of-order
CachingCollector cc = CachingCollector.create(new NoOpCollector(false), true, 100 * ONE_BYTE);
LeafCollector acc = cc.getLeafCollector(null);
acc.setScorer(new MockScorer());
for (int i = 0; i < 10; i++) acc.collect(i);
cc.replay(new NoOpCollector(true)); // this call should not fail
cc.replay(new NoOpCollector(false)); // this call should not fail
// 'src' Collector supports out-of-order
cc = CachingCollector.create(new NoOpCollector(true), true, 100 * ONE_BYTE);
acc = cc.getLeafCollector(null);
acc.setScorer(new MockScorer());
for (int i = 0; i < 10; i++) acc.collect(i);
cc.replay(new NoOpCollector(true)); // this call should not fail
try {
cc.replay(new NoOpCollector(false)); // this call should fail
fail("should have failed if an in-order Collector was given to replay(), " +
"while CachingCollector was initialized with out-of-order collection");
} catch (IllegalArgumentException e) {
// ok
}
}
public void testCachedArraysAllocation() throws Exception { public void testCachedArraysAllocation() throws Exception {
// tests the cached arrays allocation -- if the 'nextLength' was too high, // tests the cached arrays allocation -- if the 'nextLength' was too high,
// caching would terminate even if a smaller length would suffice. // caching would terminate even if a smaller length would suffice.
@ -154,7 +111,7 @@ public class TestCachingCollector extends LuceneTestCase {
int numDocs = random().nextInt(10000) + 150; int numDocs = random().nextInt(10000) + 150;
for (boolean cacheScores : new boolean[] { false, true }) { for (boolean cacheScores : new boolean[] { false, true }) {
int bytesPerDoc = cacheScores ? 8 : 4; int bytesPerDoc = cacheScores ? 8 : 4;
CachingCollector cc = CachingCollector.create(new NoOpCollector(false), CachingCollector cc = CachingCollector.create(new NoOpCollector(),
cacheScores, bytesPerDoc * ONE_BYTE * numDocs); cacheScores, bytesPerDoc * ONE_BYTE * numDocs);
LeafCollector acc = cc.getLeafCollector(null); LeafCollector acc = cc.getLeafCollector(null);
acc.setScorer(new MockScorer()); acc.setScorer(new MockScorer());
@ -176,7 +133,7 @@ public class TestCachingCollector extends LuceneTestCase {
acc.collect(0); acc.collect(0);
assertTrue(cc.isCached()); assertTrue(cc.isCached());
cc.replay(new NoOpCollector(true)); cc.replay(new NoOpCollector());
} }
} }

View File

@ -67,11 +67,6 @@ public class TestConstantScoreQuery extends LuceneTestCase {
assertEquals("Score differs from expected", expectedScore, this.scorer.score(), 0); assertEquals("Score differs from expected", expectedScore, this.scorer.score(), 0);
count[0]++; count[0]++;
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
}); });
assertEquals("invalid number of results", 1, count[0]); assertEquals("invalid number of results", 1, count[0]);
} }

View File

@ -498,7 +498,7 @@ public class TestDisjunctionMaxQuery extends LuceneTestCase {
SpanQuery sq2 = new SpanTermQuery(new Term(FIELD, "clckwork")); SpanQuery sq2 = new SpanTermQuery(new Term(FIELD, "clckwork"));
query.add(sq1); query.add(sq1);
query.add(sq2); query.add(sq2);
TopScoreDocCollector collector = TopScoreDocCollector.create(1000, true); TopScoreDocCollector collector = TopScoreDocCollector.create(1000);
searcher.search(query, collector); searcher.search(query, collector);
hits = collector.topDocs().scoreDocs.length; hits = collector.topDocs().scoreDocs.length;
for (ScoreDoc scoreDoc : collector.topDocs().scoreDocs){ for (ScoreDoc scoreDoc : collector.topDocs().scoreDocs){

View File

@ -74,10 +74,6 @@ public class TestDocBoost extends LuceneTestCase {
protected void doSetNextReader(LeafReaderContext context) throws IOException { protected void doSetNextReader(LeafReaderContext context) throws IOException {
base = context.docBase; base = context.docBase;
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
}); });
float lastScore = 0.0f; float lastScore = 0.0f;

View File

@ -60,7 +60,6 @@ public class TestEarlyTermination extends LuceneTestCase {
final IndexSearcher searcher = newSearcher(reader); final IndexSearcher searcher = newSearcher(reader);
final Collector collector = new SimpleCollector() { final Collector collector = new SimpleCollector() {
final boolean outOfOrder = random().nextBoolean();
boolean collectionTerminated = true; boolean collectionTerminated = true;
@Override @Override
@ -82,11 +81,6 @@ public class TestEarlyTermination extends LuceneTestCase {
} }
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return outOfOrder;
}
}; };
searcher.search(new MatchAllDocsQuery(), collector); searcher.search(new MatchAllDocsQuery(), collector);

View File

@ -79,7 +79,7 @@ public class TestElevationComparator extends LuceneTestCase {
new SortField(null, SortField.Type.SCORE, reversed) new SortField(null, SortField.Type.SCORE, reversed)
); );
TopDocsCollector<Entry> topCollector = TopFieldCollector.create(sort, 50, false, true, true, true); TopDocsCollector<Entry> topCollector = TopFieldCollector.create(sort, 50, false, true, true);
searcher.search(newq, null, topCollector); searcher.search(newq, null, topCollector);
TopDocs topDocs = topCollector.topDocs(0, 10); TopDocs topDocs = topCollector.topDocs(0, 10);

View File

@ -241,10 +241,6 @@ public class TestMultiTermConstantScore extends BaseTestRangeFilter {
protected void doSetNextReader(LeafReaderContext context) throws IOException { protected void doSetNextReader(LeafReaderContext context) throws IOException {
base = context.docBase; base = context.docBase;
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
}); });
// //

View File

@ -86,7 +86,7 @@ public class TestPositiveScoresOnlyCollector extends LuceneTestCase {
IndexSearcher searcher = newSearcher(ir); IndexSearcher searcher = newSearcher(ir);
Weight fake = new TermQuery(new Term("fake", "weight")).createWeight(searcher); Weight fake = new TermQuery(new Term("fake", "weight")).createWeight(searcher);
Scorer s = new SimpleScorer(fake); Scorer s = new SimpleScorer(fake);
TopDocsCollector<ScoreDoc> tdc = TopScoreDocCollector.create(scores.length, true); TopDocsCollector<ScoreDoc> tdc = TopScoreDocCollector.create(scores.length);
Collector c = new PositiveScoresOnlyCollector(tdc); Collector c = new PositiveScoresOnlyCollector(tdc);
LeafCollector ac = c.getLeafCollector(ir.leaves().get(0)); LeafCollector ac = c.getLeafCollector(ir.leaves().get(0));
ac.setScorer(s); ac.setScorer(s);

View File

@ -91,10 +91,6 @@ public class TestScoreCachingWrappingScorer extends LuceneTestCase {
this.scorer = new ScoreCachingWrappingScorer(scorer); this.scorer = new ScoreCachingWrappingScorer(scorer);
} }
@Override public boolean acceptsDocsOutOfOrder() {
return true;
}
} }
private static final float[] scores = new float[] { 0.7767749f, 1.7839992f, private static final float[] scores = new float[] { 0.7767749f, 1.7839992f,

View File

@ -115,10 +115,6 @@ public class TestScorerPerf extends LuceneTestCase {
protected void doSetNextReader(LeafReaderContext context) throws IOException { protected void doSetNextReader(LeafReaderContext context) throws IOException {
docBase = context.docBase; docBase = context.docBase;
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
} }

View File

@ -111,10 +111,6 @@ public class TestSimilarity extends LuceneTestCase {
protected void doSetNextReader(LeafReaderContext context) throws IOException { protected void doSetNextReader(LeafReaderContext context) throws IOException {
base = context.docBase; base = context.docBase;
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
}); });
PhraseQuery pq = new PhraseQuery(); PhraseQuery pq = new PhraseQuery();
@ -133,10 +129,6 @@ public class TestSimilarity extends LuceneTestCase {
//System.out.println("Doc=" + doc + " score=" + score); //System.out.println("Doc=" + doc + " score=" + score);
assertEquals(1.0f, scorer.score(), 0); assertEquals(1.0f, scorer.score(), 0);
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
}); });
pq.setSlop(2); pq.setSlop(2);
@ -152,10 +144,6 @@ public class TestSimilarity extends LuceneTestCase {
//System.out.println("Doc=" + doc + " score=" + score); //System.out.println("Doc=" + doc + " score=" + score);
assertEquals(2.0f, scorer.score(), 0); assertEquals(2.0f, scorer.score(), 0);
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
}); });
reader.close(); reader.close();

View File

@ -191,11 +191,6 @@ public class TestSloppyPhraseQuery extends LuceneTestCase {
totalHits++; totalHits++;
max = Math.max(max, scorer.freq()); max = Math.max(max, scorer.freq());
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
} }
/** checks that no scores or freqs are infinite */ /** checks that no scores or freqs are infinite */
@ -213,11 +208,6 @@ public class TestSloppyPhraseQuery extends LuceneTestCase {
assertFalse(Float.isInfinite(scorer.freq())); assertFalse(Float.isInfinite(scorer.freq()));
assertFalse(Float.isInfinite(scorer.score())); assertFalse(Float.isInfinite(scorer.score()));
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
}); });
QueryUtils.check(random(), pq, searcher); QueryUtils.check(random(), pq, searcher);
} }

View File

@ -123,8 +123,7 @@ public class TestSubScorerFreqs extends LuceneTestCase {
@Test @Test
public void testTermQuery() throws Exception { public void testTermQuery() throws Exception {
TermQuery q = new TermQuery(new Term("f", "d")); TermQuery q = new TermQuery(new Term("f", "d"));
CountingCollector c = new CountingCollector(TopScoreDocCollector.create(10, CountingCollector c = new CountingCollector(TopScoreDocCollector.create(10));
true));
s.search(q, null, c); s.search(q, null, c);
final int maxDocs = s.getIndexReader().maxDoc(); final int maxDocs = s.getIndexReader().maxDoc();
assertEquals(maxDocs, c.docCounts.size()); assertEquals(maxDocs, c.docCounts.size());
@ -164,7 +163,7 @@ public class TestSubScorerFreqs extends LuceneTestCase {
for (final Set<String> occur : occurList) { for (final Set<String> occur : occurList) {
CountingCollector c = new CountingCollector(TopScoreDocCollector.create( CountingCollector c = new CountingCollector(TopScoreDocCollector.create(
10, true), occur); 10), occur);
s.search(query, null, c); s.search(query, null, c);
final int maxDocs = s.getIndexReader().maxDoc(); final int maxDocs = s.getIndexReader().maxDoc();
assertEquals(maxDocs, c.docCounts.size()); assertEquals(maxDocs, c.docCounts.size());
@ -196,8 +195,7 @@ public class TestSubScorerFreqs extends LuceneTestCase {
PhraseQuery q = new PhraseQuery(); PhraseQuery q = new PhraseQuery();
q.add(new Term("f", "b")); q.add(new Term("f", "b"));
q.add(new Term("f", "c")); q.add(new Term("f", "c"));
CountingCollector c = new CountingCollector(TopScoreDocCollector.create(10, CountingCollector c = new CountingCollector(TopScoreDocCollector.create(10));
true));
s.search(q, null, c); s.search(q, null, c);
final int maxDocs = s.getIndexReader().maxDoc(); final int maxDocs = s.getIndexReader().maxDoc();
assertEquals(maxDocs, c.docCounts.size()); assertEquals(maxDocs, c.docCounts.size());

View File

@ -78,7 +78,7 @@ public class TestTermScorer extends LuceneTestCase {
Weight weight = indexSearcher.createNormalizedWeight(termQuery); Weight weight = indexSearcher.createNormalizedWeight(termQuery);
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, true, context.reader().getLiveDocs()); BulkScorer ts = weight.bulkScorer(context, context.reader().getLiveDocs());
// we have 2 documents with the term all in them, one document for all the // we have 2 documents with the term all in them, one document for all the
// other values // other values
final List<TestHit> docs = new ArrayList<>(); final List<TestHit> docs = new ArrayList<>();
@ -107,11 +107,6 @@ public class TestTermScorer extends LuceneTestCase {
protected void doSetNextReader(LeafReaderContext context) throws IOException { protected void doSetNextReader(LeafReaderContext context) throws IOException {
base = context.docBase; base = context.docBase;
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
}); });
assertTrue("docs Size: " + docs.size() + " is not: " + 2, docs.size() == 2); assertTrue("docs Size: " + docs.size() + " is not: " + 2, docs.size() == 2);
TestHit doc0 = docs.get(0); TestHit doc0 = docs.get(0);

View File

@ -357,11 +357,6 @@ public class TestTimeLimitingCollector extends LuceneTestCase {
docBase = context.docBase; docBase = context.docBase;
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
} }
} }

View File

@ -68,11 +68,6 @@ public class TestTopDocsCollector extends LuceneTestCase {
public void setScorer(Scorer scorer) { public void setScorer(Scorer scorer) {
// Don't do anything. Assign scores in random // Don't do anything. Assign scores in random
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
}; };
} }

View File

@ -198,7 +198,7 @@ public class TestTopDocsMerge extends LuceneTestCase {
final TopDocs topHits; final TopDocs topHits;
if (sort == null) { if (sort == null) {
if (useFrom) { if (useFrom) {
TopScoreDocCollector c = TopScoreDocCollector.create(numHits, random().nextBoolean()); TopScoreDocCollector c = TopScoreDocCollector.create(numHits);
searcher.search(query, c); searcher.search(query, c);
from = TestUtil.nextInt(random(), 0, numHits - 1); from = TestUtil.nextInt(random(), 0, numHits - 1);
size = numHits - from; size = numHits - from;
@ -217,7 +217,7 @@ public class TestTopDocsMerge extends LuceneTestCase {
topHits = searcher.search(query, numHits); topHits = searcher.search(query, numHits);
} }
} else { } else {
final TopFieldCollector c = TopFieldCollector.create(sort, numHits, true, true, true, random().nextBoolean()); final TopFieldCollector c = TopFieldCollector.create(sort, numHits, true, true, true);
searcher.search(query, c); searcher.search(query, c);
if (useFrom) { if (useFrom) {
from = TestUtil.nextInt(random(), 0, numHits - 1); from = TestUtil.nextInt(random(), 0, numHits - 1);
@ -261,7 +261,7 @@ public class TestTopDocsMerge extends LuceneTestCase {
if (sort == null) { if (sort == null) {
subHits = subSearcher.search(w, numHits); subHits = subSearcher.search(w, numHits);
} else { } else {
final TopFieldCollector c = TopFieldCollector.create(sort, numHits, true, true, true, random().nextBoolean()); final TopFieldCollector c = TopFieldCollector.create(sort, numHits, true, true, true);
subSearcher.search(w, c); subSearcher.search(w, c);
subHits = c.topDocs(0, numHits); subHits = c.topDocs(0, numHits);
} }

View File

@ -63,7 +63,7 @@ public class TestTopFieldCollector extends LuceneTestCase {
for(int i = 0; i < sort.length; i++) { for(int i = 0; i < sort.length; i++) {
Query q = new MatchAllDocsQuery(); Query q = new MatchAllDocsQuery();
TopDocsCollector<Entry> tdc = TopFieldCollector.create(sort[i], 10, false, TopDocsCollector<Entry> tdc = TopFieldCollector.create(sort[i], 10, false,
false, false, true); false, false);
is.search(q, tdc); is.search(q, tdc);
@ -82,7 +82,7 @@ public class TestTopFieldCollector extends LuceneTestCase {
for(int i = 0; i < sort.length; i++) { for(int i = 0; i < sort.length; i++) {
Query q = new MatchAllDocsQuery(); Query q = new MatchAllDocsQuery();
TopDocsCollector<Entry> tdc = TopFieldCollector.create(sort[i], 10, true, false, TopDocsCollector<Entry> tdc = TopFieldCollector.create(sort[i], 10, true, false,
false, true); false);
is.search(q, tdc); is.search(q, tdc);
@ -102,7 +102,7 @@ public class TestTopFieldCollector extends LuceneTestCase {
for(int i = 0; i < sort.length; i++) { for(int i = 0; i < sort.length; i++) {
Query q = new MatchAllDocsQuery(); Query q = new MatchAllDocsQuery();
TopDocsCollector<Entry> tdc = TopFieldCollector.create(sort[i], 10, true, true, TopDocsCollector<Entry> tdc = TopFieldCollector.create(sort[i], 10, true, true,
false, true); false);
is.search(q, tdc); is.search(q, tdc);
@ -123,7 +123,7 @@ public class TestTopFieldCollector extends LuceneTestCase {
for(int i = 0; i < sort.length; i++) { for(int i = 0; i < sort.length; i++) {
Query q = new MatchAllDocsQuery(); Query q = new MatchAllDocsQuery();
TopDocsCollector<Entry> tdc = TopFieldCollector.create(sort[i], 10, true, true, TopDocsCollector<Entry> tdc = TopFieldCollector.create(sort[i], 10, true, true,
false, true); false);
is.search(q, tdc); is.search(q, tdc);
@ -143,7 +143,7 @@ public class TestTopFieldCollector extends LuceneTestCase {
for(int i = 0; i < sort.length; i++) { for(int i = 0; i < sort.length; i++) {
Query q = new MatchAllDocsQuery(); Query q = new MatchAllDocsQuery();
TopDocsCollector<Entry> tdc = TopFieldCollector.create(sort[i], 10, true, true, TopDocsCollector<Entry> tdc = TopFieldCollector.create(sort[i], 10, true, true,
true, true); true);
is.search(q, tdc); is.search(q, tdc);
@ -156,109 +156,12 @@ public class TestTopFieldCollector extends LuceneTestCase {
} }
} }
public void testOutOfOrderDocsScoringSort() throws Exception {
// Two Sort criteria to instantiate the multi/single comparators.
Sort[] sort = new Sort[] {new Sort(SortField.FIELD_DOC), new Sort() };
boolean[][] tfcOptions = new boolean[][] {
new boolean[] { false, false, false },
new boolean[] { false, false, true },
new boolean[] { false, true, false },
new boolean[] { false, true, true },
new boolean[] { true, false, false },
new boolean[] { true, false, true },
new boolean[] { true, true, false },
new boolean[] { true, true, true },
};
String[] actualTFCClasses = new String[] {
"OutOfOrderNonScoringCollector",
"OutOfOrderScoringMaxScoreCollector",
"OutOfOrderScoringNoMaxScoreCollector",
"OutOfOrderScoringMaxScoreCollector",
"OutOfOrderNonScoringCollector",
"OutOfOrderScoringMaxScoreCollector",
"OutOfOrderScoringNoMaxScoreCollector",
"OutOfOrderScoringMaxScoreCollector"
};
BooleanQuery bq = new BooleanQuery();
// Add a Query with SHOULD, since bw.scorer() returns BooleanScorer2
// which delegates to BS if there are no mandatory clauses.
bq.add(new MatchAllDocsQuery(), Occur.SHOULD);
// Set minNrShouldMatch to 1 so that BQ will not optimize rewrite to return
// the clause instead of BQ.
bq.setMinimumNumberShouldMatch(1);
for(int i = 0; i < sort.length; i++) {
for(int j = 0; j < tfcOptions.length; j++) {
TopDocsCollector<Entry> tdc = TopFieldCollector.create(sort[i], 10,
tfcOptions[j][0], tfcOptions[j][1], tfcOptions[j][2], false);
assertTrue(tdc.getClass().getName().endsWith("$"+actualTFCClasses[j]));
is.search(bq, tdc);
TopDocs td = tdc.topDocs();
ScoreDoc[] sd = td.scoreDocs;
assertEquals(10, sd.length);
}
}
}
// OutOfOrderMulti*Collector
public void testOutOfOrderDocsScoringSortMulti() throws Exception {
// Two Sort criteria to instantiate the multi/single comparators.
Sort[] sort = new Sort[] {new Sort(SortField.FIELD_DOC, SortField.FIELD_SCORE) };
boolean[][] tfcOptions = new boolean[][] {
new boolean[] { false, false, false },
new boolean[] { false, false, true },
new boolean[] { false, true, false },
new boolean[] { false, true, true },
new boolean[] { true, false, false },
new boolean[] { true, false, true },
new boolean[] { true, true, false },
new boolean[] { true, true, true },
};
String[] actualTFCClasses = new String[] {
"OutOfOrderNonScoringCollector",
"OutOfOrderScoringMaxScoreCollector",
"OutOfOrderScoringNoMaxScoreCollector",
"OutOfOrderScoringMaxScoreCollector",
"OutOfOrderNonScoringCollector",
"OutOfOrderScoringMaxScoreCollector",
"OutOfOrderScoringNoMaxScoreCollector",
"OutOfOrderScoringMaxScoreCollector"
};
BooleanQuery bq = new BooleanQuery();
// Add a Query with SHOULD, since bw.scorer() returns BooleanScorer2
// which delegates to BS if there are no mandatory clauses.
bq.add(new MatchAllDocsQuery(), Occur.SHOULD);
// Set minNrShouldMatch to 1 so that BQ will not optimize rewrite to return
// the clause instead of BQ.
bq.setMinimumNumberShouldMatch(1);
for(int i = 0; i < sort.length; i++) {
for(int j = 0; j < tfcOptions.length; j++) {
TopDocsCollector<Entry> tdc = TopFieldCollector.create(sort[i], 10,
tfcOptions[j][0], tfcOptions[j][1], tfcOptions[j][2], false);
assertTrue(tdc.getClass().getName().endsWith("$"+actualTFCClasses[j]));
is.search(bq, tdc);
TopDocs td = tdc.topDocs();
ScoreDoc[] sd = td.scoreDocs;
assertEquals(10, sd.length);
}
}
}
public void testSortWithScoreAndMaxScoreTrackingNoResults() throws Exception { public void testSortWithScoreAndMaxScoreTrackingNoResults() throws Exception {
// Two Sort criteria to instantiate the multi/single comparators. // Two Sort criteria to instantiate the multi/single comparators.
Sort[] sort = new Sort[] {new Sort(SortField.FIELD_DOC), new Sort() }; Sort[] sort = new Sort[] {new Sort(SortField.FIELD_DOC), new Sort() };
for(int i = 0; i < sort.length; i++) { for(int i = 0; i < sort.length; i++) {
TopDocsCollector<Entry> tdc = TopFieldCollector.create(sort[i], 10, true, true, true, true); TopDocsCollector<Entry> tdc = TopFieldCollector.create(sort[i], 10, true, true, true);
TopDocs td = tdc.topDocs(); TopDocs td = tdc.topDocs();
assertEquals(0, td.totalHits); assertEquals(0, td.totalHits);
assertTrue(Float.isNaN(td.getMaxScore())); assertTrue(Float.isNaN(td.getMaxScore()));

View File

@ -1,65 +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.document.Document;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.search.BooleanClause.Occur;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.LuceneTestCase;
public class TestTopScoreDocCollector extends LuceneTestCase {
public void testOutOfOrderCollection() throws Exception {
Directory dir = newDirectory();
RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
for (int i = 0; i < 10; i++) {
writer.addDocument(new Document());
}
boolean[] inOrder = new boolean[] { false, true };
BooleanQuery bq = new BooleanQuery();
// Add a Query with SHOULD, since bw.scorer() returns BooleanScorer2
// which delegates to BS if there are no mandatory clauses.
bq.add(new MatchAllDocsQuery(), Occur.SHOULD);
// Set minNrShouldMatch to 1 so that BQ will not optimize rewrite to return
// the clause instead of BQ.
bq.setMinimumNumberShouldMatch(1);
IndexReader reader = writer.getReader();
IndexSearcher searcher = newSearcher(reader);
for (int i = 0; i < inOrder.length; i++) {
TopDocsCollector<ScoreDoc> tdc = TopScoreDocCollector.create(3, inOrder[i]);
LeafCollector leafCollector = tdc.getLeafCollector(reader.leaves().get(0));
assertEquals(!inOrder[i], leafCollector.acceptsDocsOutOfOrder());
searcher.search(new MatchAllDocsQuery(), tdc);
ScoreDoc[] sd = tdc.topDocs().scoreDocs;
assertEquals(3, sd.length);
for (int j = 0; j < sd.length; j++) {
assertEquals("expected doc Id " + j + " found " + sd[j].doc, j, sd[j].doc);
}
}
writer.close();
reader.close();
dir.close();
}
}

View File

@ -199,8 +199,7 @@ public class DrillSideways {
after, after,
true, true,
doDocScores, doDocScores,
doMaxScore, doMaxScore);
true);
DrillSidewaysResult r = search(query, hitCollector); DrillSidewaysResult r = search(query, hitCollector);
return new DrillSidewaysResult(r.facets, hitCollector.topDocs()); return new DrillSidewaysResult(r.facets, hitCollector.topDocs());
} else { } else {
@ -227,7 +226,7 @@ public class DrillSideways {
limit = 1; // the collector does not alow numHits = 0 limit = 1; // the collector does not alow numHits = 0
} }
topN = Math.min(topN, limit); topN = Math.min(topN, limit);
TopScoreDocCollector hitCollector = TopScoreDocCollector.create(topN, after, true); TopScoreDocCollector hitCollector = TopScoreDocCollector.create(topN, after);
DrillSidewaysResult r = search(query, hitCollector); DrillSidewaysResult r = search(query, hitCollector);
return new DrillSidewaysResult(r.facets, hitCollector.topDocs()); return new DrillSidewaysResult(r.facets, hitCollector.topDocs());
} }
@ -236,12 +235,7 @@ public class DrillSideways {
* (e.g., {@code ToParentBlockJoinCollector}) expects all * (e.g., {@code ToParentBlockJoinCollector}) expects all
* sub-scorers to be positioned on the document being * sub-scorers to be positioned on the document being
* collected. This will cause some performance loss; * collected. This will cause some performance loss;
* default is false. Note that if you return true from * default is false. */
* this method (in a subclass) be sure your collector
* also returns false from {@link
* LeafCollector#acceptsDocsOutOfOrder}: this will trick
* {@code BooleanQuery} into also scoring all subDocs at
* once. */
protected boolean scoreSubDocsAtOnce() { protected boolean scoreSubDocsAtOnce() {
return false; return false;
} }

View File

@ -110,13 +110,6 @@ class DrillSidewaysQuery extends Query {
baseWeight.normalize(norm, topLevelBoost); baseWeight.normalize(norm, topLevelBoost);
} }
@Override
public boolean scoresDocsOutOfOrder() {
// TODO: would be nice if AssertingIndexSearcher
// confirmed this for us
return false;
}
@Override @Override
public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException { public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
// We can only run as a top scorer: // We can only run as a top scorer:
@ -124,7 +117,7 @@ class DrillSidewaysQuery extends Query {
} }
@Override @Override
public BulkScorer bulkScorer(LeafReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) throws IOException { public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
// TODO: it could be better if we take acceptDocs // TODO: it could be better if we take acceptDocs
// into account instead of baseScorer? // into account instead of baseScorer?

View File

@ -154,14 +154,6 @@ public class FacetsCollector extends SimpleCollector {
return matchingDocs; return matchingDocs;
} }
@Override
public final boolean acceptsDocsOutOfOrder() {
// If we are keeping scores then we require in-order
// because we append each score to the float[] and
// expect that they correlate in order to the hits:
return keepScores == false;
}
@Override @Override
public final void collect(int doc) throws IOException { public final void collect(int doc) throws IOException {
docs.addDoc(doc); docs.addDoc(doc);
@ -284,14 +276,9 @@ public class FacetsCollector extends SimpleCollector {
(FieldDoc) after, (FieldDoc) after,
fillFields, fillFields,
doDocScores, doDocScores,
doMaxScore, doMaxScore);
false);
} else { } else {
// TODO: can we pass the right boolean for hitsCollector = TopScoreDocCollector.create(n, after);
// in-order instead of hardwired to false...? we'd
// need access to the protected IS.search methods
// taking Weight... could use reflection...
hitsCollector = TopScoreDocCollector.create(n, after, false);
} }
searcher.search(q, MultiCollector.wrap(hitsCollector, fc)); searcher.search(q, MultiCollector.wrap(hitsCollector, fc));
return hitsCollector.topDocs(); return hitsCollector.topDocs();

View File

@ -54,9 +54,4 @@ class AssertingSubDocsAtOnceCollector extends SimpleCollector {
} }
} }
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
} }

View File

@ -682,11 +682,6 @@ public class TestDrillSideways extends FacetTestCase {
protected void doSetNextReader(LeafReaderContext context) throws IOException { protected void doSetNextReader(LeafReaderContext context) throws IOException {
lastDocID = -1; lastDocID = -1;
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
}); });
// Also separately verify that DS respects the // Also separately verify that DS respects the

View File

@ -126,11 +126,6 @@ public abstract class AbstractAllGroupHeadsCollector<GH extends AbstractAllGroup
groupHead.updateDocHead(doc); groupHead.updateDocHead(doc);
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
/** /**
* Contains the result of group head retrieval. * Contains the result of group head retrieval.
* To prevent new object creations of this class for every collect. * To prevent new object creations of this class for every collect.

View File

@ -62,8 +62,4 @@ public abstract class AbstractAllGroupsCollector<GROUP_VALUE_TYPE> extends Simpl
@Override @Override
public void setScorer(Scorer scorer) throws IOException {} public void setScorer(Scorer scorer) throws IOException {}
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
} }

View File

@ -37,11 +37,6 @@ public abstract class AbstractDistinctValuesCollector<GC extends AbstractDistinc
*/ */
public abstract List<GC> getGroups(); public abstract List<GC> getGroups();
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
/** /**
* Returned by {@link AbstractDistinctValuesCollector#getGroups()}, * Returned by {@link AbstractDistinctValuesCollector#getGroups()},
* representing the value and set of distinct values for the group. * representing the value and set of distinct values for the group.

View File

@ -320,11 +320,6 @@ abstract public class AbstractFirstPassGroupingCollector<GROUP_VALUE_TYPE> exten
} }
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
@Override @Override
protected void doSetNextReader(LeafReaderContext readerContext) throws IOException { protected void doSetNextReader(LeafReaderContext readerContext) throws IOException {
docBase = readerContext.docBase; docBase = readerContext.docBase;

View File

@ -109,11 +109,6 @@ public abstract class AbstractGroupFacetCollector extends SimpleCollector {
public void setScorer(Scorer scorer) throws IOException { public void setScorer(Scorer scorer) throws IOException {
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
/** /**
* The grouped facet result. Containing grouped facet entries, total count and total missing count. * The grouped facet result. Containing grouped facet entries, total count and total missing count.
*/ */

View File

@ -69,10 +69,10 @@ public abstract class AbstractSecondPassGroupingCollector<GROUP_VALUE_TYPE> exte
final TopDocsCollector<?> collector; final TopDocsCollector<?> collector;
if (withinGroupSort == null) { if (withinGroupSort == null) {
// Sort by score // Sort by score
collector = TopScoreDocCollector.create(maxDocsPerGroup, true); collector = TopScoreDocCollector.create(maxDocsPerGroup);
} else { } else {
// Sort by fields // Sort by fields
collector = TopFieldCollector.create(withinGroupSort, maxDocsPerGroup, fillSortFields, getScores, getMaxScores, true); collector = TopFieldCollector.create(withinGroupSort, maxDocsPerGroup, fillSortFields, getScores, getMaxScores);
} }
groupMap.put(group.groupValue, groupMap.put(group.groupValue,
new SearchGroupDocs<>(group.groupValue, new SearchGroupDocs<>(group.groupValue,
@ -114,11 +114,6 @@ public abstract class AbstractSecondPassGroupingCollector<GROUP_VALUE_TYPE> exte
} }
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
public TopGroups<GROUP_VALUE_TYPE> getTopGroups(int withinGroupOffset) { public TopGroups<GROUP_VALUE_TYPE> getTopGroups(int withinGroupOffset) {
@SuppressWarnings({"unchecked","rawtypes"}) @SuppressWarnings({"unchecked","rawtypes"})
final GroupDocs<GROUP_VALUE_TYPE>[] groupDocsResult = (GroupDocs<GROUP_VALUE_TYPE>[]) new GroupDocs[groups.size()]; final GroupDocs<GROUP_VALUE_TYPE>[] groupDocsResult = (GroupDocs<GROUP_VALUE_TYPE>[]) new GroupDocs[groups.size()];

View File

@ -345,10 +345,10 @@ public class BlockGroupingCollector extends SimpleCollector {
if (!needsScores) { if (!needsScores) {
throw new IllegalArgumentException("cannot sort by relevance within group: needsScores=false"); throw new IllegalArgumentException("cannot sort by relevance within group: needsScores=false");
} }
collector = TopScoreDocCollector.create(maxDocsPerGroup, true); collector = TopScoreDocCollector.create(maxDocsPerGroup);
} else { } else {
// Sort by fields // Sort by fields
collector = TopFieldCollector.create(withinGroupSort, maxDocsPerGroup, fillSortFields, needsScores, needsScores, true); collector = TopFieldCollector.create(withinGroupSort, maxDocsPerGroup, fillSortFields, needsScores, needsScores);
} }
LeafCollector leafCollector = collector.getLeafCollector(og.readerContext); LeafCollector leafCollector = collector.getLeafCollector(og.readerContext);
@ -512,11 +512,6 @@ public class BlockGroupingCollector extends SimpleCollector {
} }
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
@Override @Override
protected void doSetNextReader(LeafReaderContext readerContext) throws IOException { protected void doSetNextReader(LeafReaderContext readerContext) throws IOException {
if (subDocUpto != 0) { if (subDocUpto != 0) {

View File

@ -119,11 +119,6 @@ public class HighlighterPhraseTest extends LuceneTestCase {
indexSearcher.search(phraseQuery, new SimpleCollector() { indexSearcher.search(phraseQuery, new SimpleCollector() {
private int baseDoc; private int baseDoc;
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
@Override @Override
public void collect(int i) { public void collect(int i) {
bitset.set(this.baseDoc + i); bitset.set(this.baseDoc + i);

View File

@ -45,11 +45,6 @@ abstract class TermsCollector extends SimpleCollector {
return collectorTerms; return collectorTerms;
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
/** /**
* Chooses the right {@link TermsCollector} implementation. * Chooses the right {@link TermsCollector} implementation.
* *

View File

@ -133,20 +133,24 @@ class TermsIncludingScoreQuery extends Query {
@Override @Override
public Explanation explain(LeafReaderContext context, int doc) throws IOException { public Explanation explain(LeafReaderContext context, int doc) throws IOException {
SVInnerScorer scorer = (SVInnerScorer) bulkScorer(context, false, null); Terms terms = context.reader().terms(field);
if (scorer != null) { if (terms != null) {
return scorer.explain(doc); segmentTermsEnum = terms.iterator(segmentTermsEnum);
BytesRef spare = new BytesRef();
DocsEnum docsEnum = null;
for (int i = 0; i < TermsIncludingScoreQuery.this.terms.size(); i++) {
if (segmentTermsEnum.seekExact(TermsIncludingScoreQuery.this.terms.get(ords[i], spare))) {
docsEnum = segmentTermsEnum.docs(null, docsEnum, DocsEnum.FLAG_NONE);
if (docsEnum.advance(doc) == doc) {
final float score = TermsIncludingScoreQuery.this.scores[ords[i]];
return new ComplexExplanation(true, score, "Score based on join value " + segmentTermsEnum.term().utf8ToString());
}
}
}
} }
return new ComplexExplanation(false, 0.0f, "Not a match"); return new ComplexExplanation(false, 0.0f, "Not a match");
} }
@Override
public boolean scoresDocsOutOfOrder() {
// We have optimized impls below if we are allowed
// to score out-of-order:
return true;
}
@Override @Override
public Query getQuery() { public Query getQuery() {
return TermsIncludingScoreQuery.this; return TermsIncludingScoreQuery.this;
@ -179,143 +183,9 @@ class TermsIncludingScoreQuery extends Query {
return new SVInOrderScorer(this, acceptDocs, segmentTermsEnum, context.reader().maxDoc(), cost); return new SVInOrderScorer(this, acceptDocs, segmentTermsEnum, context.reader().maxDoc(), cost);
} }
} }
@Override
public BulkScorer bulkScorer(LeafReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) throws IOException {
if (scoreDocsInOrder) {
return super.bulkScorer(context, scoreDocsInOrder, acceptDocs);
} else {
Terms terms = context.reader().terms(field);
if (terms == null) {
return null;
}
// what is the runtime...seems ok?
final long cost = context.reader().maxDoc() * terms.size();
segmentTermsEnum = terms.iterator(segmentTermsEnum);
// Optimized impls that take advantage of docs
// being allowed to be out of order:
if (multipleValuesPerDocument) {
return new MVInnerScorer(this, acceptDocs, segmentTermsEnum, context.reader().maxDoc(), cost);
} else {
return new SVInnerScorer(this, acceptDocs, segmentTermsEnum, cost);
}
}
}
}; };
} }
// This impl assumes that the 'join' values are used uniquely per doc per field. Used for one to many relations.
class SVInnerScorer extends BulkScorer {
final BytesRef spare = new BytesRef();
final Bits acceptDocs;
final TermsEnum termsEnum;
final long cost;
int upto;
DocsEnum docsEnum;
DocsEnum reuse;
int scoreUpto;
int doc;
SVInnerScorer(Weight weight, Bits acceptDocs, TermsEnum termsEnum, long cost) {
this.acceptDocs = acceptDocs;
this.termsEnum = termsEnum;
this.cost = cost;
this.doc = -1;
}
@Override
public boolean score(LeafCollector collector, int max) throws IOException {
FakeScorer fakeScorer = new FakeScorer();
collector.setScorer(fakeScorer);
if (doc == -1) {
doc = nextDocOutOfOrder();
}
while(doc < max) {
fakeScorer.doc = doc;
fakeScorer.score = scores[ords[scoreUpto]];
collector.collect(doc);
doc = nextDocOutOfOrder();
}
return doc != DocsEnum.NO_MORE_DOCS;
}
int nextDocOutOfOrder() throws IOException {
while (true) {
if (docsEnum != null) {
int docId = docsEnumNextDoc();
if (docId == DocIdSetIterator.NO_MORE_DOCS) {
docsEnum = null;
} else {
return doc = docId;
}
}
if (upto == terms.size()) {
return doc = DocIdSetIterator.NO_MORE_DOCS;
}
scoreUpto = upto;
if (termsEnum.seekExact(terms.get(ords[upto++], spare))) {
docsEnum = reuse = termsEnum.docs(acceptDocs, reuse, DocsEnum.FLAG_NONE);
}
}
}
protected int docsEnumNextDoc() throws IOException {
return docsEnum.nextDoc();
}
private Explanation explain(int target) throws IOException {
int docId;
do {
docId = nextDocOutOfOrder();
if (docId < target) {
int tempDocId = docsEnum.advance(target);
if (tempDocId == target) {
docId = tempDocId;
break;
}
} else if (docId == target) {
break;
}
docsEnum = null; // goto the next ord.
} while (docId != DocIdSetIterator.NO_MORE_DOCS);
return new ComplexExplanation(true, scores[ords[scoreUpto]], "Score based on join value " + termsEnum.term().utf8ToString());
}
}
// This impl that tracks whether a docid has already been emitted. This check makes sure that docs aren't emitted
// twice for different join values. This means that the first encountered join value determines the score of a document
// even if other join values yield a higher score.
class MVInnerScorer extends SVInnerScorer {
final FixedBitSet alreadyEmittedDocs;
MVInnerScorer(Weight weight, Bits acceptDocs, TermsEnum termsEnum, int maxDoc, long cost) {
super(weight, acceptDocs, termsEnum, cost);
alreadyEmittedDocs = new FixedBitSet(maxDoc);
}
@Override
protected int docsEnumNextDoc() throws IOException {
while (true) {
int docId = docsEnum.nextDoc();
if (docId == DocIdSetIterator.NO_MORE_DOCS) {
return docId;
}
if (!alreadyEmittedDocs.getAndSet(docId)) {
return docId;//if it wasn't previously set, return it
}
}
}
}
class SVInOrderScorer extends Scorer { class SVInOrderScorer extends Scorer {
final DocIdSetIterator matchingDocsIterator; final DocIdSetIterator matchingDocsIterator;

View File

@ -57,11 +57,6 @@ abstract class TermsWithScoreCollector extends SimpleCollector {
this.scorer = scorer; this.scorer = scorer;
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
/** /**
* Chooses the right {@link TermsWithScoreCollector} implementation. * Chooses the right {@link TermsWithScoreCollector} implementation.
* *

View File

@ -154,11 +154,6 @@ public class ToChildBlockJoinQuery extends Query {
throw new UnsupportedOperationException(getClass().getName() + throw new UnsupportedOperationException(getClass().getName() +
" cannot explain match on parent document"); " cannot explain match on parent document");
} }
@Override
public boolean scoresDocsOutOfOrder() {
return false;
}
} }
static class ToChildBlockJoinScorer extends Scorer { static class ToChildBlockJoinScorer extends Scorer {

View File

@ -38,6 +38,9 @@ import java.util.*;
* IndexWriter#updateDocuments}). Ie, the join is computed * IndexWriter#updateDocuments}). Ie, the join is computed
* at index time. * at index time.
* *
* <p>This collector MUST be used with {@link ToParentBlockJoinIndexSearcher},
* in order to work correctly.
*
* <p>The parent Sort must only use * <p>The parent Sort must only use
* fields from the parent documents; sorting by field in * fields from the parent documents; sorting by field in
* the child documents is not supported.</p> * the child documents is not supported.</p>
@ -296,11 +299,6 @@ public class ToParentBlockJoinCollector implements Collector {
} }
} }
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
}; };
} }
@ -408,10 +406,10 @@ public class ToParentBlockJoinCollector implements Collector {
if (!trackScores) { if (!trackScores) {
throw new IllegalArgumentException("cannot sort by relevance within group: trackScores=false"); throw new IllegalArgumentException("cannot sort by relevance within group: trackScores=false");
} }
collector = TopScoreDocCollector.create(numDocsInGroup, true); collector = TopScoreDocCollector.create(numDocsInGroup);
} else { } else {
// Sort by fields // Sort by fields
collector = TopFieldCollector.create(withinGroupSort, numDocsInGroup, fillSortFields, trackScores, trackMaxScore, true); collector = TopFieldCollector.create(withinGroupSort, numDocsInGroup, fillSortFields, trackScores, trackMaxScore);
} }
LeafCollector leafCollector = collector.getLeafCollector(og.readerContext); LeafCollector leafCollector = collector.getLeafCollector(og.readerContext);

View File

@ -0,0 +1,69 @@
package org.apache.lucene.search.join;
/*
* 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 java.io.IOException;
import java.util.List;
import java.util.concurrent.ExecutorService;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.Collector;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.LeafCollector;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.Weight;
/**
* An {@link IndexSearcher} to use in conjunction with
* {@link ToParentBlockJoinCollector}.
*/
public class ToParentBlockJoinIndexSearcher extends IndexSearcher {
/** Creates a searcher searching the provided index. Search on individual
* segments will be run in the provided {@link ExecutorService}.
* @see IndexSearcher#IndexSearcher(IndexReader, ExecutorService) */
public ToParentBlockJoinIndexSearcher(IndexReader r, ExecutorService executor) {
super(r, executor);
}
/** Creates a searcher searching the provided index.
* @see IndexSearcher#IndexSearcher(IndexReader) */
public ToParentBlockJoinIndexSearcher(IndexReader r) {
super(r);
}
@Override
protected void search(List<LeafReaderContext> leaves, Weight weight, Collector collector) throws IOException {
for (LeafReaderContext ctx : leaves) { // search each subreader
// we force the use of Scorer (not BulkScorer) to make sure
// that the scorer passed to LeafCollector.setScorer supports
// Scorer.getChildren
Scorer scorer = weight.scorer(ctx, ctx.reader().getLiveDocs());
if (scorer != null) {
final LeafCollector leafCollector = collector.getLeafCollector(ctx);
leafCollector.setScorer(scorer);
for (int doc = scorer.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = scorer.nextDoc()) {
leafCollector.collect(doc);
}
}
}
}
}

View File

@ -194,11 +194,6 @@ public class ToParentBlockJoinQuery extends Query {
} }
return new ComplexExplanation(false, 0.0f, "Not a match"); return new ComplexExplanation(false, 0.0f, "Not a match");
} }
@Override
public boolean scoresDocsOutOfOrder() {
return false;
}
} }
static class BlockJoinScorer extends Scorer { static class BlockJoinScorer extends Scorer {

View File

@ -1314,6 +1314,7 @@ public class TestBlockJoin extends LuceneTestCase {
IndexReader r = w.getReader(); IndexReader r = w.getReader();
w.close(); w.close();
IndexSearcher searcher = new ToParentBlockJoinIndexSearcher(r);
Query childQuery = new TermQuery(new Term("childText", "text")); Query childQuery = new TermQuery(new Term("childText", "text"));
BitDocIdSetFilter parentsFilter = new BitDocIdSetCachingWrapperFilter(new QueryWrapperFilter(new TermQuery(new Term("isParent", "yes")))); BitDocIdSetFilter parentsFilter = new BitDocIdSetCachingWrapperFilter(new QueryWrapperFilter(new TermQuery(new Term("isParent", "yes"))));
ToParentBlockJoinQuery childJoinQuery = new ToParentBlockJoinQuery(childQuery, parentsFilter, ScoreMode.Avg); ToParentBlockJoinQuery childJoinQuery = new ToParentBlockJoinQuery(childQuery, parentsFilter, ScoreMode.Avg);
@ -1323,7 +1324,7 @@ public class TestBlockJoin extends LuceneTestCase {
ToParentBlockJoinCollector c = new ToParentBlockJoinCollector(new Sort(new SortField("parentID", SortField.Type.STRING)), ToParentBlockJoinCollector c = new ToParentBlockJoinCollector(new Sort(new SortField("parentID", SortField.Type.STRING)),
10, true, true); 10, true, true);
newSearcher(r).search(parentQuery, c); searcher.search(parentQuery, c);
TopGroups<Integer> groups = c.getTopGroups(childJoinQuery, null, 0, 10, 0, false); TopGroups<Integer> groups = c.getTopGroups(childJoinQuery, null, 0, 10, 0, false);
// Two parents: // Two parents:
@ -1381,6 +1382,8 @@ public class TestBlockJoin extends LuceneTestCase {
IndexReader r = w.getReader(); IndexReader r = w.getReader();
w.close(); w.close();
IndexSearcher searcher = new ToParentBlockJoinIndexSearcher(r);
// never matches: // never matches:
Query childQuery = new TermQuery(new Term("childText", "bogus")); Query childQuery = new TermQuery(new Term("childText", "bogus"));
BitDocIdSetFilter parentsFilter = new BitDocIdSetCachingWrapperFilter(new QueryWrapperFilter(new TermQuery(new Term("isParent", "yes")))); BitDocIdSetFilter parentsFilter = new BitDocIdSetCachingWrapperFilter(new QueryWrapperFilter(new TermQuery(new Term("isParent", "yes"))));
@ -1391,7 +1394,7 @@ public class TestBlockJoin extends LuceneTestCase {
ToParentBlockJoinCollector c = new ToParentBlockJoinCollector(new Sort(new SortField("parentID", SortField.Type.STRING)), ToParentBlockJoinCollector c = new ToParentBlockJoinCollector(new Sort(new SortField("parentID", SortField.Type.STRING)),
10, true, true); 10, true, true);
newSearcher(r).search(parentQuery, c); searcher.search(parentQuery, c);
TopGroups<Integer> groups = c.getTopGroups(childJoinQuery, null, 0, 10, 0, false); TopGroups<Integer> groups = c.getTopGroups(childJoinQuery, null, 0, 10, 0, false);
// Two parents: // Two parents:

View File

@ -310,10 +310,6 @@ public class TestJoinUtil extends LuceneTestCase {
assertFalse("optimized bulkScorer was not used for join query embedded in boolean query!", sawFive); assertFalse("optimized bulkScorer was not used for join query embedded in boolean query!", sawFive);
} }
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
}); });
indexSearcher.getIndexReader().close(); indexSearcher.getIndexReader().close();
@ -447,8 +443,7 @@ public class TestJoinUtil extends LuceneTestCase {
dir, dir,
newIndexWriterConfig(new MockAnalyzer(random(), MockTokenizer.KEYWORD, false)).setMergePolicy(newLogMergePolicy()) newIndexWriterConfig(new MockAnalyzer(random(), MockTokenizer.KEYWORD, false)).setMergePolicy(newLogMergePolicy())
); );
final boolean scoreDocsInOrder = TestJoinUtil.random().nextBoolean(); IndexIterationContext context = createContext(numberOfDocumentsToIndex, w, multipleValuesPerDocument);
IndexIterationContext context = createContext(numberOfDocumentsToIndex, w, multipleValuesPerDocument, scoreDocsInOrder);
IndexReader topLevelReader = w.getReader(); IndexReader topLevelReader = w.getReader();
w.close(); w.close();
@ -484,7 +479,7 @@ public class TestJoinUtil extends LuceneTestCase {
// Need to know all documents that have matches. TopDocs doesn't give me that and then I'd be also testing TopDocsCollector... // Need to know all documents that have matches. TopDocs doesn't give me that and then I'd be also testing TopDocsCollector...
final BitSet actualResult = new FixedBitSet(indexSearcher.getIndexReader().maxDoc()); final BitSet actualResult = new FixedBitSet(indexSearcher.getIndexReader().maxDoc());
final TopScoreDocCollector topScoreDocCollector = TopScoreDocCollector.create(10, false); final TopScoreDocCollector topScoreDocCollector = TopScoreDocCollector.create(10);
indexSearcher.search(joinQuery, new Collector() { indexSearcher.search(joinQuery, new Collector() {
@Override @Override
@ -498,11 +493,6 @@ public class TestJoinUtil extends LuceneTestCase {
super.collect(doc); super.collect(doc);
actualResult.set(doc + docBase); actualResult.set(doc + docBase);
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return scoreDocsInOrder;
}
}; };
} }
}); });
@ -547,11 +537,11 @@ public class TestJoinUtil extends LuceneTestCase {
} }
} }
private IndexIterationContext createContext(int nDocs, RandomIndexWriter writer, boolean multipleValuesPerDocument, boolean scoreDocsInOrder) throws IOException { private IndexIterationContext createContext(int nDocs, RandomIndexWriter writer, boolean multipleValuesPerDocument) throws IOException {
return createContext(nDocs, writer, writer, multipleValuesPerDocument, scoreDocsInOrder); return createContext(nDocs, writer, writer, multipleValuesPerDocument);
} }
private IndexIterationContext createContext(int nDocs, RandomIndexWriter fromWriter, RandomIndexWriter toWriter, boolean multipleValuesPerDocument, boolean scoreDocsInOrder) throws IOException { private IndexIterationContext createContext(int nDocs, RandomIndexWriter fromWriter, RandomIndexWriter toWriter, boolean multipleValuesPerDocument) throws IOException {
IndexIterationContext context = new IndexIterationContext(); IndexIterationContext context = new IndexIterationContext();
int numRandomValues = nDocs / 2; int numRandomValues = nDocs / 2;
context.randomUniqueValues = new String[numRandomValues]; context.randomUniqueValues = new String[numRandomValues];
@ -683,11 +673,6 @@ public class TestJoinUtil extends LuceneTestCase {
public void setScorer(Scorer scorer) { public void setScorer(Scorer scorer) {
this.scorer = scorer; this.scorer = scorer;
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
}); });
} else { } else {
fromSearcher.search(new TermQuery(new Term("value", uniqueRandomValue)), new SimpleCollector() { fromSearcher.search(new TermQuery(new Term("value", uniqueRandomValue)), new SimpleCollector() {
@ -720,17 +705,11 @@ public class TestJoinUtil extends LuceneTestCase {
public void setScorer(Scorer scorer) { public void setScorer(Scorer scorer) {
this.scorer = scorer; this.scorer = scorer;
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
}); });
} }
final Map<Integer, JoinScore> docToJoinScore = new HashMap<>(); final Map<Integer, JoinScore> docToJoinScore = new HashMap<>();
if (multipleValuesPerDocument) { if (multipleValuesPerDocument) {
if (scoreDocsInOrder) {
LeafReader slowCompositeReader = SlowCompositeReaderWrapper.wrap(toSearcher.getIndexReader()); LeafReader slowCompositeReader = SlowCompositeReaderWrapper.wrap(toSearcher.getIndexReader());
Terms terms = slowCompositeReader.terms(toField); Terms terms = slowCompositeReader.terms(toField);
if (terms != null) { if (terms != null) {
@ -757,43 +736,6 @@ public class TestJoinUtil extends LuceneTestCase {
} else { } else {
toSearcher.search(new MatchAllDocsQuery(), new SimpleCollector() { toSearcher.search(new MatchAllDocsQuery(), new SimpleCollector() {
private SortedSetDocValues docTermOrds;
private int docBase;
@Override
public void collect(int doc) throws IOException {
docTermOrds.setDocument(doc);
long ord;
while ((ord = docTermOrds.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
final BytesRef joinValue = docTermOrds.lookupOrd(ord);
JoinScore joinScore = joinValueToJoinScores.get(joinValue);
if (joinScore == null) {
continue;
}
Integer basedDoc = docBase + doc;
// First encountered join value determines the score.
// Something to keep in mind for many-to-many relations.
if (!docToJoinScore.containsKey(basedDoc)) {
docToJoinScore.put(basedDoc, joinScore);
}
}
}
@Override
protected void doSetNextReader(LeafReaderContext context) throws IOException {
docBase = context.docBase;
docTermOrds = DocValues.getSortedSet(context.reader(), toField);
}
@Override
public boolean acceptsDocsOutOfOrder() {return false;}
@Override
public void setScorer(Scorer scorer) {}
});
}
} else {
toSearcher.search(new MatchAllDocsQuery(), new SimpleCollector() {
private BinaryDocValues terms; private BinaryDocValues terms;
private int docBase; private int docBase;
@ -813,8 +755,6 @@ public class TestJoinUtil extends LuceneTestCase {
docBase = context.docBase; docBase = context.docBase;
} }
@Override
public boolean acceptsDocsOutOfOrder() {return false;}
@Override @Override
public void setScorer(Scorer scorer) {} public void setScorer(Scorer scorer) {}
}); });

View File

@ -569,11 +569,6 @@ public class MemoryIndex {
this.scorer = scorer; this.scorer = scorer;
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
}); });
float score = scores[0]; float score = scores[0];
return score; return score;

View File

@ -128,11 +128,6 @@ public class EarlyTerminatingSortingCollector extends FilterCollector {
} }
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
}; };
} else { } else {
return super.getLeafCollector(context); return super.getLeafCollector(context);

View File

@ -126,9 +126,8 @@ public class TestEarlyTerminatingSortingCollector extends LuceneTestCase {
final boolean fillFields = random().nextBoolean(); final boolean fillFields = random().nextBoolean();
final boolean trackDocScores = random().nextBoolean(); final boolean trackDocScores = random().nextBoolean();
final boolean trackMaxScore = random().nextBoolean(); final boolean trackMaxScore = random().nextBoolean();
final boolean inOrder = random().nextBoolean(); final TopFieldCollector collector1 = TopFieldCollector.create(sort, numHits, fillFields, trackDocScores, trackMaxScore);
final TopFieldCollector collector1 = TopFieldCollector.create(sort, numHits, fillFields, trackDocScores, trackMaxScore, inOrder); final TopFieldCollector collector2 = TopFieldCollector.create(sort, numHits, fillFields, trackDocScores, trackMaxScore);
final TopFieldCollector collector2 = TopFieldCollector.create(sort, numHits, fillFields, trackDocScores, trackMaxScore, inOrder);
final Query query; final Query query;
if (random().nextBoolean()) { if (random().nextBoolean()) {
@ -191,9 +190,8 @@ public class TestEarlyTerminatingSortingCollector extends LuceneTestCase {
final boolean fillFields = random().nextBoolean(); final boolean fillFields = random().nextBoolean();
final boolean trackDocScores = random().nextBoolean(); final boolean trackDocScores = random().nextBoolean();
final boolean trackMaxScore = random().nextBoolean(); final boolean trackMaxScore = random().nextBoolean();
final boolean inOrder = random().nextBoolean(); final TopFieldCollector collector1 = TopFieldCollector.create(sort, numHits, fillFields, trackDocScores, trackMaxScore);
final TopFieldCollector collector1 = TopFieldCollector.create(sort, numHits, fillFields, trackDocScores, trackMaxScore, inOrder); final TopFieldCollector collector2 = TopFieldCollector.create(sort, numHits, fillFields, trackDocScores, trackMaxScore);
final TopFieldCollector collector2 = TopFieldCollector.create(sort, numHits, fillFields, trackDocScores, trackMaxScore, inOrder);
final Query query; final Query query;
if (random().nextBoolean()) { if (random().nextBoolean()) {

View File

@ -271,11 +271,6 @@ public class CustomScoreQuery extends Query {
return res; return res;
} }
@Override
public boolean scoresDocsOutOfOrder() {
return false;
}
} }

View File

@ -71,11 +71,6 @@ public class BooleanQueryTst {
this.scorer = scorer; this.scorer = scorer;
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
@Override @Override
protected void doSetNextReader(LeafReaderContext context) throws IOException { protected void doSetNextReader(LeafReaderContext context) throws IOException {
docBase = context.docBase; docBase = context.docBase;

View File

@ -289,15 +289,13 @@ public class TestTermAutomatonQuery extends LuceneTestCase {
s.search(q, new SimpleCollector() { s.search(q, new SimpleCollector() {
private Scorer scorer; private Scorer scorer;
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
@Override @Override
public void setScorer(Scorer scorer) { public void setScorer(Scorer scorer) {
assert scorer instanceof TermAutomatonScorer;
this.scorer = scorer; this.scorer = scorer;
while (scorer instanceof AssertingScorer) {
scorer = ((AssertingScorer) scorer).getIn();
}
assert scorer instanceof TermAutomatonScorer;
} }
@Override @Override

View File

@ -547,7 +547,7 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
//System.out.println("finalQuery=" + query); //System.out.println("finalQuery=" + query);
// Sort by weight, descending: // Sort by weight, descending:
TopFieldCollector c = TopFieldCollector.create(SORT, num, true, false, false, false); TopFieldCollector c = TopFieldCollector.create(SORT, num, true, false, false);
// We sorted postings by weight during indexing, so we // We sorted postings by weight during indexing, so we
// only retrieve the first num hits now: // only retrieve the first num hits now:

View File

@ -18,21 +18,12 @@ package org.apache.lucene.search;
*/ */
import java.io.IOException; import java.io.IOException;
import java.lang.ref.WeakReference;
import java.util.Collection;
import java.util.Collections;
import java.util.Map;
import java.util.Random; import java.util.Random;
import java.util.WeakHashMap;
import org.apache.lucene.index.DocsEnum; import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.util.VirtualMethod;
/** Wraps a Scorer with additional checks */ /** Wraps a Scorer with additional checks */
public class AssertingBulkScorer extends BulkScorer { final class AssertingBulkScorer extends BulkScorer {
private static final VirtualMethod<BulkScorer> SCORE_COLLECTOR = new VirtualMethod<>(BulkScorer.class, "score", LeafCollector.class);
private static final VirtualMethod<BulkScorer> SCORE_COLLECTOR_RANGE = new VirtualMethod<>(BulkScorer.class, "score", LeafCollector.class, int.class);
public static BulkScorer wrap(Random random, BulkScorer other) { public static BulkScorer wrap(Random random, BulkScorer other) {
if (other == null || other instanceof AssertingBulkScorer) { if (other == null || other instanceof AssertingBulkScorer) {
@ -41,10 +32,6 @@ public class AssertingBulkScorer extends BulkScorer {
return new AssertingBulkScorer(random, other); return new AssertingBulkScorer(random, other);
} }
public static boolean shouldWrap(BulkScorer inScorer) {
return SCORE_COLLECTOR.isOverriddenAsOf(inScorer.getClass()) || SCORE_COLLECTOR_RANGE.isOverriddenAsOf(inScorer.getClass());
}
final Random random; final Random random;
final BulkScorer in; final BulkScorer in;
@ -59,6 +46,7 @@ public class AssertingBulkScorer extends BulkScorer {
@Override @Override
public void score(LeafCollector collector) throws IOException { public void score(LeafCollector collector) throws IOException {
collector = new AssertingLeafCollector(random, collector, DocsEnum.NO_MORE_DOCS);
if (random.nextBoolean()) { if (random.nextBoolean()) {
try { try {
final boolean remaining = in.score(collector, DocsEnum.NO_MORE_DOCS); final boolean remaining = in.score(collector, DocsEnum.NO_MORE_DOCS);
@ -73,6 +61,7 @@ public class AssertingBulkScorer extends BulkScorer {
@Override @Override
public boolean score(LeafCollector collector, int max) throws IOException { public boolean score(LeafCollector collector, int max) throws IOException {
collector = new AssertingLeafCollector(random, collector, max);
return in.score(collector, max); return in.score(collector, max);
} }

View File

@ -1,67 +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 java.io.IOException;
import java.util.Random;
import org.apache.lucene.index.LeafReaderContext;
/** Wraps another Collector and checks that
* acceptsDocsOutOfOrder is respected. */
public class AssertingCollector extends FilterCollector {
public static Collector wrap(Random random, Collector other, boolean inOrder) {
return other instanceof AssertingCollector ? other : new AssertingCollector(random, other, inOrder);
}
final Random random;
final boolean inOrder;
AssertingCollector(Random random, Collector in, boolean inOrder) {
super(in);
this.random = random;
this.inOrder = inOrder;
}
@Override
public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
return new FilterLeafCollector(super.getLeafCollector(context)) {
int lastCollected = -1;
@Override
public void setScorer(Scorer scorer) throws IOException {
super.setScorer(AssertingScorer.getAssertingScorer(random, scorer));
}
@Override
public void collect(int doc) throws IOException {
if (inOrder || !acceptsDocsOutOfOrder()) {
assert doc > lastCollected : "Out of order : " + lastCollected + " " + doc;
}
in.collect(doc);
lastCollected = doc;
}
};
}
}

View File

@ -20,35 +20,37 @@ package org.apache.lucene.search;
import java.io.IOException; import java.io.IOException;
import java.util.Random; import java.util.Random;
/** A crazy {@link BulkScorer} that wraps another {@link BulkScorer} /** Wraps another Collector and checks that
* but shuffles the order of the collected documents. */ * acceptsDocsOutOfOrder is respected. */
public class AssertingBulkOutOfOrderScorer extends BulkScorer {
final BulkScorer in; final class AssertingLeafCollector extends FilterLeafCollector {
final Random random;
public AssertingBulkOutOfOrderScorer(Random random, BulkScorer in) { private final Random random;
this.in = in; private final int max;
private Scorer scorer;
private int lastCollected = -1;
AssertingLeafCollector(Random random, LeafCollector collector, int max) {
super(collector);
this.random = random; this.random = random;
this.max = max;
} }
@Override @Override
public boolean score(LeafCollector collector, int max) throws IOException { public void setScorer(Scorer scorer) throws IOException {
final RandomOrderCollector randomCollector = new RandomOrderCollector(random, collector); this.scorer = scorer;
final boolean remaining = in.score(randomCollector, max); super.setScorer(AssertingScorer.getAssertingScorer(random, scorer));
randomCollector.flush();
return remaining;
} }
@Override @Override
public void score(LeafCollector collector) throws IOException { public void collect(int doc) throws IOException {
final RandomOrderCollector randomCollector = new RandomOrderCollector(random, collector); assert doc > lastCollected : "Out of order : " + lastCollected + " " + doc;
in.score(randomCollector); assert doc < max : "Out of range: " + doc + " >= " + max;
randomCollector.flush(); assert scorer.docID() == doc : "Collected: " + doc + " but scorer: " + scorer.docID();
in.collect(doc);
lastCollected = doc;
} }
@Override
public String toString() {
return "AssertingBulkOutOfOrderScorer(" + in + ")";
}
} }

View File

@ -29,14 +29,12 @@ class AssertingWeight extends Weight {
return other instanceof AssertingWeight ? other : new AssertingWeight(random, other); return other instanceof AssertingWeight ? other : new AssertingWeight(random, other);
} }
final boolean scoresDocsOutOfOrder;
final Random random; final Random random;
final Weight in; final Weight in;
AssertingWeight(Random random, Weight in) { AssertingWeight(Random random, Weight in) {
this.random = random; this.random = random;
this.in = in; this.in = in;
scoresDocsOutOfOrder = in.scoresDocsOutOfOrder() || random.nextBoolean();
} }
@Override @Override
@ -68,36 +66,12 @@ class AssertingWeight extends Weight {
} }
@Override @Override
public BulkScorer bulkScorer(LeafReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) throws IOException { public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
// if the caller asks for in-order scoring or if the weight does not support BulkScorer inScorer = in.bulkScorer(context, acceptDocs);
// out-of order scoring then collection will have to happen in-order.
BulkScorer inScorer = in.bulkScorer(context, scoreDocsInOrder, acceptDocs);
if (inScorer == null) { if (inScorer == null) {
return null; return null;
} }
if (AssertingBulkScorer.shouldWrap(inScorer)) { return AssertingBulkScorer.wrap(new Random(random.nextLong()), inScorer);
// The incoming scorer already has a specialized
// implementation for BulkScorer, so we should use it:
inScorer = AssertingBulkScorer.wrap(new Random(random.nextLong()), inScorer);
} else if (random.nextBoolean()) {
// Let super wrap this.scorer instead, so we use
// AssertingScorer:
inScorer = super.bulkScorer(context, scoreDocsInOrder, acceptDocs);
}
if (scoreDocsInOrder == false && random.nextBoolean()) {
// The caller claims it can handle out-of-order
// docs; let's confirm that by pulling docs and
// randomly shuffling them before collection:
inScorer = new AssertingBulkOutOfOrderScorer(new Random(random.nextLong()), inScorer);
}
return inScorer;
}
@Override
public boolean scoresDocsOutOfOrder() {
return scoresDocsOutOfOrder;
} }
} }

View File

@ -139,10 +139,6 @@ public class CheckHits {
protected void doSetNextReader(LeafReaderContext context) throws IOException { protected void doSetNextReader(LeafReaderContext context) throws IOException {
base = context.docBase; base = context.docBase;
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
} }
/** /**
@ -511,10 +507,6 @@ public class CheckHits {
protected void doSetNextReader(LeafReaderContext context) throws IOException { protected void doSetNextReader(LeafReaderContext context) throws IOException {
base = context.docBase; base = context.docBase;
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
} }
} }

View File

@ -219,7 +219,6 @@ public class QueryUtils {
public static void checkSkipTo(final Query q, final IndexSearcher s) throws IOException { public static void checkSkipTo(final Query q, final IndexSearcher s) throws IOException {
//System.out.println("Checking "+q); //System.out.println("Checking "+q);
final List<LeafReaderContext> readerContextArray = s.getTopReaderContext().leaves(); final List<LeafReaderContext> readerContextArray = s.getTopReaderContext().leaves();
if (s.createNormalizedWeight(q).scoresDocsOutOfOrder()) return; // in this case order of skipTo() might differ from that of next().
final int skip_op = 0; final int skip_op = 0;
final int next_op = 1; final int next_op = 1;
@ -323,11 +322,6 @@ public class QueryUtils {
this.scorer = null; this.scorer = null;
lastDoc[0] = -1; lastDoc[0] = -1;
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
}); });
if (lastReader[0] != null) { if (lastReader[0] != null) {
@ -409,10 +403,6 @@ public class QueryUtils {
lastDoc[0] = -1; lastDoc[0] = -1;
liveDocs = context.reader().getLiveDocs(); liveDocs = context.reader().getLiveDocs();
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
}); });
if (lastReader[0] != null) { if (lastReader[0] != null) {

View File

@ -1,106 +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 java.io.IOException;
import java.util.Random;
/** Randomize collection order. Don't forget to call {@link #flush()} when
* collection is finished to collect buffered documents. */
final class RandomOrderCollector extends FilterLeafCollector {
final Random random;
Scorer scorer;
FakeScorer fakeScorer;
int buffered;
final int bufferSize;
final int[] docIDs;
final float[] scores;
final int[] freqs;
RandomOrderCollector(Random random, LeafCollector in) {
super(in);
if (!in.acceptsDocsOutOfOrder()) {
throw new IllegalArgumentException();
}
this.random = random;
bufferSize = 1 + random.nextInt(100);
docIDs = new int[bufferSize];
scores = new float[bufferSize];
freqs = new int[bufferSize];
buffered = 0;
}
@Override
public void setScorer(Scorer scorer) throws IOException {
this.scorer = scorer;
fakeScorer = new FakeScorer();
in.setScorer(fakeScorer);
}
private void shuffle() {
for (int i = buffered - 1; i > 0; --i) {
final int other = random.nextInt(i + 1);
final int tmpDoc = docIDs[i];
docIDs[i] = docIDs[other];
docIDs[other] = tmpDoc;
final float tmpScore = scores[i];
scores[i] = scores[other];
scores[other] = tmpScore;
final int tmpFreq = freqs[i];
freqs[i] = freqs[other];
freqs[other] = tmpFreq;
}
}
public void flush() throws IOException {
shuffle();
for (int i = 0; i < buffered; ++i) {
fakeScorer.doc = docIDs[i];
fakeScorer.freq = freqs[i];
fakeScorer.score = scores[i];
in.collect(fakeScorer.doc);
}
buffered = 0;
}
@Override
public void collect(int doc) throws IOException {
docIDs[buffered] = doc;
scores[buffered] = scorer.score();
try {
freqs[buffered] = scorer.freq();
} catch (UnsupportedOperationException e) {
freqs[buffered] = -1;
}
if (++buffered == bufferSize) {
flush();
}
}
@Override
public boolean acceptsDocsOutOfOrder() {
return in.acceptsDocsOutOfOrder();
}
}

View File

@ -38,9 +38,4 @@ public abstract class ValueAccumulator extends SimpleCollector {
// NOP // NOP
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
} }

View File

@ -497,7 +497,7 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
DocIdSetIterator iterator = new BitSetIterator(groupBits, 0); // cost is not useful here DocIdSetIterator iterator = new BitSetIterator(groupBits, 0); // cost is not useful here
int group; int group;
while ((group = iterator.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) { while ((group = iterator.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
Collector collector = (sort == null) ? TopScoreDocCollector.create(limit, true) : TopFieldCollector.create(sort, limit, false, false, false, true); Collector collector = (sort == null) ? TopScoreDocCollector.create(limit) : TopFieldCollector.create(sort, limit, false, false, false);
groups.put(group, collector); groups.put(group, collector);
} }
@ -549,11 +549,6 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
c.collect(docId); c.collect(docId);
} }
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
}; };
} }
@ -579,7 +574,7 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
Iterator<LongCursor> iterator = groupSet.iterator(); Iterator<LongCursor> iterator = groupSet.iterator();
while (iterator.hasNext()) { while (iterator.hasNext()) {
LongCursor cursor = iterator.next(); LongCursor cursor = iterator.next();
Collector collector = (sort == null) ? TopScoreDocCollector.create(limit, true) : TopFieldCollector.create(sort, limit, false, false, false, true); Collector collector = (sort == null) ? TopScoreDocCollector.create(limit) : TopFieldCollector.create(sort, limit, false, false, false);
groups.put(cursor.value, collector); groups.put(cursor.value, collector);
} }
@ -614,11 +609,6 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
c.collect(docId); c.collect(docId);
} }
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
}; };
} }

View File

@ -450,12 +450,6 @@ public class CollapsingQParserPlugin extends QParserPlugin {
} }
} }
@Override
public boolean acceptsDocsOutOfOrder() {
//Documents must be sent in order to this collector.
return false;
}
@Override @Override
protected void doSetNextReader(LeafReaderContext context) throws IOException { protected void doSetNextReader(LeafReaderContext context) throws IOException {
this.contexts[context.ord] = context; this.contexts[context.ord] = context;
@ -665,12 +659,6 @@ public class CollapsingQParserPlugin extends QParserPlugin {
} }
@Override
public boolean acceptsDocsOutOfOrder() {
//Documents must be sent in order to this collector.
return false;
}
@Override @Override
protected void doSetNextReader(LeafReaderContext context) throws IOException { protected void doSetNextReader(LeafReaderContext context) throws IOException {
this.contexts[context.ord] = context; this.contexts[context.ord] = context;

View File

@ -75,11 +75,6 @@ public class DelegatingCollector extends SimpleCollector {
leafDelegate = delegate.getLeafCollector(context); leafDelegate = delegate.getLeafCollector(context);
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return leafDelegate.acceptsDocsOutOfOrder();
}
public void finish() throws IOException { public void finish() throws IOException {
if(delegate instanceof DelegatingCollector) { if(delegate instanceof DelegatingCollector) {
((DelegatingCollector) delegate).finish(); ((DelegatingCollector) delegate).finish();

View File

@ -87,9 +87,4 @@ public class DocSetCollector extends SimpleCollector {
protected void doSetNextReader(LeafReaderContext context) throws IOException { protected void doSetNextReader(LeafReaderContext context) throws IOException {
this.base = context.docBase; this.base = context.docBase;
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
} }

View File

@ -63,16 +63,6 @@ public class EarlyTerminatingCollector extends FilterCollector {
return new FilterLeafCollector(super.getLeafCollector(context)) { return new FilterLeafCollector(super.getLeafCollector(context)) {
/**
* This collector requires that docs be collected in order, otherwise
* the computed number of scanned docs in the resulting
* {@link EarlyTerminatingCollectorException} will be meaningless.
*/
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
@Override @Override
public void collect(int doc) throws IOException { public void collect(int doc) throws IOException {
super.collect(doc); super.collect(doc);

View File

@ -148,11 +148,6 @@ public class ExportQParserPlugin extends QParserPlugin {
++totalHits; ++totalHits;
set.set(docId); set.set(docId);
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
}; };
} }

View File

@ -879,9 +879,9 @@ public class Grouping {
TopDocsCollector newCollector(Sort sort, boolean needScores) throws IOException { TopDocsCollector newCollector(Sort sort, boolean needScores) throws IOException {
int groupDocsToCollect = getMax(groupOffset, docsPerGroup, maxDoc); int groupDocsToCollect = getMax(groupOffset, docsPerGroup, maxDoc);
if (sort == null || sort == Sort.RELEVANCE) { if (sort == null || sort == Sort.RELEVANCE) {
return TopScoreDocCollector.create(groupDocsToCollect, true); return TopScoreDocCollector.create(groupDocsToCollect);
} else { } else {
return TopFieldCollector.create(searcher.weightSort(sort), groupDocsToCollect, false, needScores, needScores, true); return TopFieldCollector.create(searcher.weightSort(sort), groupDocsToCollect, false, needScores, needScores);
} }
} }

View File

@ -248,10 +248,10 @@ public class ReRankQParserPlugin extends QParserPlugin {
this.boostedPriority = boostedPriority; this.boostedPriority = boostedPriority;
Sort sort = cmd.getSort(); Sort sort = cmd.getSort();
if(sort == null) { if(sort == null) {
this.mainCollector = TopScoreDocCollector.create(Math.max(this.reRankDocs, length),true); this.mainCollector = TopScoreDocCollector.create(Math.max(this.reRankDocs, length));
} else { } else {
sort = sort.rewrite(searcher); sort = sort.rewrite(searcher);
this.mainCollector = TopFieldCollector.create(sort, Math.max(this.reRankDocs, length), false, true, true, true); this.mainCollector = TopFieldCollector.create(sort, Math.max(this.reRankDocs, length), false, true, true);
} }
this.searcher = searcher; this.searcher = searcher;
this.reRankWeight = reRankWeight; this.reRankWeight = reRankWeight;

View File

@ -1597,7 +1597,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable,SolrIn
if (null == cmd.getSort()) { if (null == cmd.getSort()) {
assert null == cmd.getCursorMark() : "have cursor but no sort"; assert null == cmd.getCursorMark() : "have cursor but no sort";
return TopScoreDocCollector.create(len, true); return TopScoreDocCollector.create(len);
} else { } else {
// we have a sort // we have a sort
final boolean needScores = (cmd.getFlags() & GET_SCORES) != 0; final boolean needScores = (cmd.getFlags() & GET_SCORES) != 0;
@ -1609,7 +1609,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable,SolrIn
final boolean fillFields = (null != cursor); final boolean fillFields = (null != cursor);
final FieldDoc searchAfter = (null != cursor ? cursor.getSearchAfterFieldDoc() : null); final FieldDoc searchAfter = (null != cursor ? cursor.getSearchAfterFieldDoc() : null);
return TopFieldCollector.create(weightedSort, len, searchAfter, return TopFieldCollector.create(weightedSort, len, searchAfter,
fillFields, needScores, needScores, true); fillFields, needScores, needScores);
} }
} }
@ -1644,10 +1644,6 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable,SolrIn
public void collect(int doc) { public void collect(int doc) {
numHits[0]++; numHits[0]++;
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
}; };
} else { } else {
collector = new SimpleCollector() { collector = new SimpleCollector() {
@ -1662,10 +1658,6 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable,SolrIn
float score = scorer.score(); float score = scorer.score();
if (score > topscore[0]) topscore[0]=score; if (score > topscore[0]) topscore[0]=score;
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
}; };
} }
@ -1750,11 +1742,6 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable,SolrIn
float score = scorer.score(); float score = scorer.score();
if (score > topscore[0]) topscore[0] = score; if (score > topscore[0]) topscore[0] = score;
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
}; };
collector = MultiCollector.wrap(setCollector, topScoreCollector); collector = MultiCollector.wrap(setCollector, topScoreCollector);

View File

@ -126,9 +126,9 @@ public class QueryCommand implements Command<QueryCommandResult> {
@Override @Override
public List<Collector> create() throws IOException { public List<Collector> create() throws IOException {
if (sort == null || sort == Sort.RELEVANCE) { if (sort == null || sort == Sort.RELEVANCE) {
collector = TopScoreDocCollector.create(docsToCollect, true); collector = TopScoreDocCollector.create(docsToCollect);
} else { } else {
collector = TopFieldCollector.create(sort, docsToCollect, true, needScores, needScores, true); collector = TopFieldCollector.create(sort, docsToCollect, true, needScores, needScores);
} }
filterCollector = new FilterCollector(docSet, collector); filterCollector = new FilterCollector(docSet, collector);
return Arrays.asList((Collector) filterCollector); return Arrays.asList((Collector) filterCollector);

View File

@ -785,11 +785,6 @@ public class TestRankQueryPlugin extends QParserPlugin {
public void collect(int doc) throws IOException { public void collect(int doc) throws IOException {
list.add(new ScoreDoc(doc+base, scorer.score())); list.add(new ScoreDoc(doc+base, scorer.score()));
} }
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
}; };
} }

View File

@ -270,7 +270,7 @@ public class TestSort extends SolrTestCaseJ4 {
boolean trackScores = r.nextBoolean(); boolean trackScores = r.nextBoolean();
boolean trackMaxScores = r.nextBoolean(); boolean trackMaxScores = r.nextBoolean();
boolean scoreInOrder = r.nextBoolean(); boolean scoreInOrder = r.nextBoolean();
final TopFieldCollector topCollector = TopFieldCollector.create(sort, top, true, trackScores, trackMaxScores, scoreInOrder); final TopFieldCollector topCollector = TopFieldCollector.create(sort, top, true, trackScores, trackMaxScores);
final List<MyDoc> collectedDocs = new ArrayList<>(); final List<MyDoc> collectedDocs = new ArrayList<>();
// delegate and collect docs ourselves // delegate and collect docs ourselves