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
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
* 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:
TopFieldCollector collector = TopFieldCollector.create(sort, numHits,
true, withScore(),
withMaxScore(),
false);
withMaxScore());
searcher.search(q, null, collector);
hits = collector.topDocs();
} else {
@ -191,7 +190,7 @@ public abstract class ReadTask extends PerfTask {
}
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
protected Collector createCollector() throws Exception {
Collector collector = null;
if (clnName.equalsIgnoreCase("topScoreDocOrdered") == true) {
collector = TopScoreDocCollector.create(numHits(), true);
} else if (clnName.equalsIgnoreCase("topScoreDocUnOrdered") == true) {
collector = TopScoreDocCollector.create(numHits(), false);
if (clnName.equalsIgnoreCase("topScoreDoc") == true) {
collector = TopScoreDocCollector.create(numHits());
} else if (clnName.length() > 0){
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.Set;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.BooleanClause.Occur;
import org.apache.lucene.search.similarities.Similarity;
@ -305,21 +305,19 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
}
@Override
public BulkScorer bulkScorer(LeafReaderContext context, boolean scoreDocsInOrder,
Bits acceptDocs) throws IOException {
public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
if (scoreDocsInOrder || minNrShouldMatch > 1) {
if (minNrShouldMatch > 1) {
// TODO: (LUCENE-4872) in some cases BooleanScorer may be faster for minNrShouldMatch
// 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>();
Iterator<BooleanClause> cIter = clauses.iterator();
for (Weight w : weights) {
BooleanClause c = cIter.next();
BulkScorer subScorer = w.bulkScorer(context, false, acceptDocs);
BulkScorer subScorer = w.bulkScorer(context, acceptDocs);
if (subScorer == null) {
if (c.isRequired()) {
return null;
@ -328,16 +326,20 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
// TODO: there are some cases where BooleanScorer
// would handle conjunctions faster than
// BooleanScorer2...
return super.bulkScorer(context, scoreDocsInOrder, acceptDocs);
return super.bulkScorer(context, acceptDocs);
} else if (c.isProhibited()) {
// TODO: there are some cases where BooleanScorer could do this faster
return super.bulkScorer(context, scoreDocsInOrder, acceptDocs);
return super.bulkScorer(context, acceptDocs);
} else {
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
@ -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) {
if (required.size() == 1) {
Scorer req = required.get(0);

View File

@ -18,75 +18,41 @@ package org.apache.lucene.search;
*/
import java.io.IOException;
import java.util.List;
import java.util.Arrays;
import java.util.Collection;
import org.apache.lucene.search.BooleanQuery.BooleanWeight;
/* Description from Doug Cutting (excerpted from
* LUCENE-1483):
*
* 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. */
/**
* BulkSorer that is used for pure disjunctions: no MUST clauses and
* minShouldMatch == 1. This scorer scores documents by batches of 2048 docs.
*/
final class BooleanScorer extends BulkScorer {
private static final class BooleanScorerCollector extends SimpleCollector {
private BucketTable bucketTable;
private int mask;
private Scorer scorer;
static final int SHIFT = 11;
static final int SIZE = 1 << SHIFT;
static final int MASK = SIZE - 1;
static final int SET_SIZE = 1 << (SHIFT - 6);
static final int SET_MASK = SET_SIZE - 1;
public BooleanScorerCollector(int mask, BucketTable bucketTable) {
this.mask = mask;
this.bucketTable = bucketTable;
static class Bucket {
double score;
int freq;
}
@Override
public void collect(final int doc) throws IOException {
final BucketTable table = bucketTable;
final int i = doc & BucketTable.MASK;
final Bucket bucket = table.buckets[i];
final Bucket[] buckets = new Bucket[SIZE];
// This is basically an inlined FixedBitSet... seems to help with bound checks
final long[] matching = new long[SET_SIZE];
if (bucket.doc != doc) { // invalid bucket
bucket.doc = doc; // set doc
bucket.score = scorer.score(); // initialize score
bucket.bits = mask; // initialize mask
bucket.coord = 1; // initialize coord
final float[] coordFactors;
final BulkScorer[] optionalScorers;
final FakeScorer fakeScorer = new FakeScorer();
bucket.next = table.first; // push onto valid list
table.first = bucket;
} else { // valid bucket
bucket.score += scorer.score(); // increment score
bucket.bits |= mask; // add bits in mask
bucket.coord++; // increment coord
}
}
boolean hasMatches;
int max = 0;
final class OrCollector implements LeafCollector {
Scorer scorer;
@Override
public void setScorer(Scorer scorer) {
@ -94,95 +60,24 @@ final class BooleanScorer extends BulkScorer {
}
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
}
static final class Bucket {
int doc = -1; // tells if bucket is valid
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 void collect(int doc) throws IOException {
hasMatches = true;
final int i = doc & MASK;
final int idx = i >>> 6;
matching[idx] |= 1L << i;
final Bucket bucket = buckets[i];
bucket.freq++;
bucket.score += scorer.score();
}
}
public LeafCollector newCollector(int mask) {
return new BooleanScorerCollector(mask, this);
}
public int size() { return SIZE; }
}
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);
final OrCollector orCollector = new OrCollector();
BooleanScorer(BooleanWeight weight, boolean disableCoord, int maxCoord, Collection<BulkScorer> optionalScorers) {
for (int i = 0; i < buckets.length; i++) {
buckets[i] = new Bucket();
}
this.optionalScorers = optionalScorers.toArray(new BulkScorer[0]);
coordFactors = new float[optionalScorers.size() + 1];
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
public boolean score(LeafCollector collector, int max) throws IOException {
fakeScorer.doc = -1;
collector.setScorer(fakeScorer);
boolean more;
Bucket tmp;
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);
for (int docBase = this.max & ~MASK; docBase < max; docBase += SIZE) {
if (scoreWindow(collector, docBase, max) == false) {
return false;
}
}
@Override
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();
return true;
}
}

View File

@ -82,7 +82,6 @@ public abstract class CachingCollector extends FilterCollector {
private static class NoScoreCachingCollector extends CachingCollector {
List<Boolean> acceptDocsOutOfOrders;
List<LeafReaderContext> contexts;
List<int[]> docs;
int maxDocsToCache;
@ -92,7 +91,6 @@ public abstract class CachingCollector extends FilterCollector {
super(in);
this.maxDocsToCache = maxDocsToCache;
contexts = new ArrayList<>();
acceptDocsOutOfOrders = new ArrayList<>();
docs = new ArrayList<>();
}
@ -105,7 +103,6 @@ public abstract class CachingCollector extends FilterCollector {
final LeafCollector in = this.in.getLeafCollector(context);
if (contexts != null) {
contexts.add(context);
acceptDocsOutOfOrders.add(in.acceptsDocsOutOfOrder());
}
if (maxDocsToCache >= 0) {
return lastCollector = wrap(in, maxDocsToCache);
@ -152,14 +149,7 @@ public abstract class CachingCollector extends FilterCollector {
assert docs.size() == contexts.size();
for (int i = 0; i < contexts.size(); ++i) {
final LeafReaderContext context = contexts.get(i);
final boolean docsInOrder = !acceptDocsOutOfOrders.get(i);
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);
}
}
@ -306,10 +296,6 @@ public abstract class CachingCollector extends FilterCollector {
*/
public static CachingCollector create(final boolean acceptDocsOutOfOrder, boolean cacheScores, double maxRAMMB) {
Collector other = new SimpleCollector() {
@Override
public boolean acceptsDocsOutOfOrder() {
return acceptDocsOutOfOrder;
}
@Override
public void collect(int doc) {}

View File

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

View File

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

View File

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

View File

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

View File

@ -98,22 +98,4 @@ public interface LeafCollector {
*/
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
@Override
public abstract boolean acceptsDocsOutOfOrder();
@Override
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();
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:
int hitUpto = 0;

View File

@ -27,7 +27,7 @@ import org.apache.lucene.util.PriorityQueue;
* A {@link Collector} that sorts by {@link SortField} using
* {@link FieldComparator}s.
* <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.
*
* @lucene.experimental
@ -55,11 +55,6 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
this.scorer = scorer;
comparator.setScorer(scorer);
}
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
}
private static abstract class MultiComparatorLeafCollector implements LeafCollector {
@ -124,11 +119,6 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
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
* 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
* 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.
*/
@ -774,8 +458,8 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
// Fastmatch: return if this hit is no better than
// the worst hit currently in the queue:
final int cmp = compareBottom(doc);
if (cmp < 0 || (cmp == 0 && doc + docBase > bottom.doc)) {
// Definitely not competitive.
if (cmp <= 0) {
// not competitive since documents are visited in doc id order
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
* each result. Also, setting this true automatically sets
* <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
* the sort criteria.
* @throws IOException if there is a low-level I/O error
*/
public static TopFieldCollector create(Sort sort, int numHits,
boolean fillFields, boolean trackDocScores, boolean trackMaxScore,
boolean docsScoredInOrder)
boolean fillFields, boolean trackDocScores, boolean trackMaxScore)
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
* each result. Also, setting this true automatically sets
* <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
* the sort criteria.
* @throws IOException if there is a low-level I/O error
*/
public static TopFieldCollector create(Sort sort, int numHits, FieldDoc after,
boolean fillFields, boolean trackDocScores, boolean trackMaxScore,
boolean docsScoredInOrder)
boolean fillFields, boolean trackDocScores, boolean trackMaxScore)
throws IOException {
if (sort.fields.length == 0) {
@ -950,7 +621,6 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
FieldValueHitQueue<Entry> queue = FieldValueHitQueue.create(sort.fields, numHits);
if (after == null) {
if (docsScoredInOrder) {
if (trackMaxScore) {
return new ScoringMaxScoreCollector(queue, numHits, fillFields);
} else if (trackDocScores) {
@ -958,15 +628,6 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
} else {
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 {
if (after.fields == null) {
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> {
private abstract static class ScorerLeafCollector implements LeafCollector {
final boolean scoreDocsInOrder;
ScorerLeafCollector(boolean scoreDocsInOrder) {
this.scoreDocsInOrder = scoreDocsInOrder;
}
abstract static class ScorerLeafCollector implements LeafCollector {
Scorer scorer;
@ -51,28 +45,19 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
this.scorer = scorer;
}
@Override
public boolean acceptsDocsOutOfOrder() {
return scoreDocsInOrder == false;
}
}
private static class SimpleTopScoreDocCollector extends TopScoreDocCollector {
private final boolean scoreDocsInOrder;
SimpleTopScoreDocCollector(int numHits, boolean scoreDocsInOrder) {
SimpleTopScoreDocCollector(int numHits) {
super(numHits);
this.scoreDocsInOrder = scoreDocsInOrder;
}
@Override
public LeafCollector getLeafCollector(LeafReaderContext context)
throws IOException {
final int docBase = context.docBase;
if (scoreDocsInOrder) {
return new ScorerLeafCollector(scoreDocsInOrder) {
return new ScorerLeafCollector() {
@Override
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 final boolean scoreDocsInOrder;
private final ScoreDoc after;
private int collectedHits;
PagingTopScoreDocCollector(int numHits, boolean scoreDocsInOrder, ScoreDoc after) {
PagingTopScoreDocCollector(int numHits, ScoreDoc after) {
super(numHits);
this.scoreDocsInOrder = scoreDocsInOrder;
this.after = after;
this.collectedHits = 0;
}
@ -153,8 +109,7 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
final int docBase = context.docBase;
final int afterDoc = after.doc - context.docBase;
if (scoreDocsInOrder) {
return new ScorerLeafCollector(scoreDocsInOrder) {
return new ScorerLeafCollector() {
@Override
public void collect(int doc) throws IOException {
float score = scorer.score();
@ -182,36 +137,6 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
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
* objects.
*/
public static TopScoreDocCollector create(int numHits, boolean docsScoredInOrder) {
return create(numHits, null, docsScoredInOrder);
public static TopScoreDocCollector create(int numHits) {
return create(numHits, null);
}
/**
@ -239,23 +164,23 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
* <code>numHits</code>, and fill the array with sentinel
* objects.
*/
public static TopScoreDocCollector create(int numHits, ScoreDoc after, boolean docsScoredInOrder) {
public static TopScoreDocCollector create(int numHits, ScoreDoc after) {
if (numHits <= 0) {
throw new IllegalArgumentException("numHits must be > 0; please use TotalHitCountCollector if you just need the total hit count");
}
if (after == null) {
return new SimpleTopScoreDocCollector(numHits, docsScoredInOrder);
return new SimpleTopScoreDocCollector(numHits);
} else {
return new PagingTopScoreDocCollector(numHits, docsScoredInOrder, after);
return new PagingTopScoreDocCollector(numHits, after);
}
}
ScoreDoc pqTop;
// prevents instantiation
private TopScoreDocCollector(int numHits) {
TopScoreDocCollector(int numHits) {
super(new HitQueue(numHits, true));
// HitQueue implements getSentinelObject to return a ScoreDoc, so we know
// that at this point top() is already initialized.

View File

@ -34,9 +34,4 @@ public class TotalHitCountCollector extends SimpleCollector {
public void collect(int doc) {
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
* to <code>scoreDocsInOrder</code>.
* <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
* query.
*
@ -108,15 +103,6 @@ public abstract class Weight {
*
* @param context
* 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
* Bits that represent the allowable docs to match (typically deleted docs
* but possibly filtering other documents)
@ -125,7 +111,7 @@ public abstract class Weight {
* passes them to a collector.
* @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);
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.
</li>
<li>
{@link org.apache.lucene.search.Weight#bulkScorer(org.apache.lucene.index.LeafReaderContext, boolean, org.apache.lucene.util.Bits)
scorer(LeafReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs)} &mdash;
{@link org.apache.lucene.search.Weight#bulkScorer(org.apache.lucene.index.LeafReaderContext, org.apache.lucene.util.Bits)
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>
below for help defining a BulkScorer. This is an optional method, and most queries do not implement it.
</li>

View File

@ -437,10 +437,6 @@ public class TestOmitTf extends LuceneTestCase {
protected void doSetNextReader(LeafReaderContext context) throws IOException {
docBase = context.docBase;
}
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
}
/** 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);
}
@Override
public boolean acceptsDocsOutOfOrder() {
throw new UnsupportedOperationException(UNSUPPORTED_MSG);
}
}
static final class JustCompileDocIdSet extends DocIdSet {

View File

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

View File

@ -130,11 +130,14 @@ public class TestBoolean2 extends LuceneTestCase {
};
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);
ScoreDoc[] hits1 = collector.topDocs().scoreDocs;
collector = TopScoreDocCollector.create(1000, true);
collector = TopScoreDocCollector.create(1000);
searcher.search(query, null, collector);
ScoreDoc[] hits2 = collector.topDocs().scoreDocs;
@ -282,13 +285,13 @@ public class TestBoolean2 extends LuceneTestCase {
}
TopFieldCollector collector = TopFieldCollector.create(sort, 1000,
false, true, true, true);
false, true, true);
searcher.search(q1, null, collector);
ScoreDoc[] hits1 = collector.topDocs().scoreDocs;
collector = TopFieldCollector.create(sort, 1000,
false, true, true, false);
false, true, true);
searcher.search(q1, null, collector);
ScoreDoc[] hits2 = collector.topDocs().scoreDocs;

View File

@ -712,10 +712,9 @@ public class TestBooleanCoord extends LuceneTestCase {
assertEquals(0, scorer.nextDoc());
assertEquals(expected, scorer.score(), 0.0001f);
// test out-of-order (if supported)
if (weight.scoresDocsOutOfOrder()) {
// test bulk scorer
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);
bulkScorer.score(new LeafCollector() {
Scorer scorer;
@ -732,14 +731,8 @@ public class TestBooleanCoord extends LuceneTestCase {
assertEquals(expected, scorer.score(), 0.0001f);
seen.set(true);
}
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
}, 1);
assertTrue(seen.get());
}
// test the explanation
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);
//System.out.println("TEST: now check");
// bs2
TopScoreDocCollector collector = TopScoreDocCollector.create(1000, true);
TopScoreDocCollector collector = TopScoreDocCollector.create(1000);
s.search(q, collector);
ScoreDoc[] h2 = collector.topDocs().scoreDocs;
if (expected != h2.length) {

View File

@ -182,7 +182,7 @@ public class TestBooleanOr extends LuceneTestCase {
Weight w = s.createNormalizedWeight(bq);
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 AtomicInteger end = new AtomicInteger();
@ -193,11 +193,6 @@ public class TestBooleanOr extends LuceneTestCase {
assertTrue("collected doc=" + doc + " beyond max=" + end, doc < end.intValue());
hits.set(doc);
}
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
};
while (end.intValue() < docCount) {

View File

@ -314,7 +314,7 @@ public class TestBooleanQuery extends LuceneTestCase {
SpanQuery sq2 = new SpanTermQuery(new Term(FIELD, "clckwork"));
query.add(sq1, BooleanClause.Occur.SHOULD);
query.add(sq2, BooleanClause.Occur.SHOULD);
TopScoreDocCollector collector = TopScoreDocCollector.create(1000, true);
TopScoreDocCollector collector = TopScoreDocCollector.create(1000);
searcher.search(query, collector);
hits = collector.topDocs().scoreDocs.length;
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.Term;
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.Weight.DefaultBulkScorer;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.LuceneTestCase;
// TODO: refactor to a base class, that collects freqs from the scorer tree
// and test all queries with it
public class TestBooleanQueryVisitSubscorers extends LuceneTestCase {
Analyzer analyzer;
IndexReader reader;
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"));
reader = writer.getReader();
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
@ -75,7 +81,7 @@ public class TestBooleanQueryVisitSubscorers extends LuceneTestCase {
}
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(F2, "lucene")), 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 {
BooleanQuery bq = new BooleanQuery();
BooleanQuery2 bq = new BooleanQuery2();
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, "search")), 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<>();
MyCollector() {
super(TopScoreDocCollector.create(10, true));
super(TopScoreDocCollector.create(10));
}
public LeafCollector getLeafCollector(LeafReaderContext context)
@ -138,11 +144,6 @@ public class TestBooleanQueryVisitSubscorers extends LuceneTestCase {
final int docBase = context.docBase;
return new FilterLeafCollector(super.getLeafCollector(context)) {
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
@Override
public void setScorer(Scorer scorer) throws IOException {
super.setScorer(scorer);
@ -205,7 +206,7 @@ public class TestBooleanQueryVisitSubscorers extends LuceneTestCase {
}
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, "miss")), Occur.SHOULD);
ScorerSummarizingCollector collector = new ScorerSummarizingCollector();
@ -246,11 +247,6 @@ public class TestBooleanQueryVisitSubscorers extends LuceneTestCase {
public void collect(int doc) throws IOException {
numHits[0]++;
}
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
};
}
@ -276,4 +272,22 @@ public class TestBooleanQueryVisitSubscorers extends LuceneTestCase {
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();
}
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 */
private static class CrazyMustUseBulkScorerQuery extends Query {
@ -162,7 +102,7 @@ public class TestBooleanScorer extends LuceneTestCase {
}
@Override
public BulkScorer bulkScorer(LeafReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) {
public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs) {
return new BulkScorer() {
@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 final boolean acceptDocsOutOfOrder;
public NoOpCollector(boolean acceptDocsOutOfOrder) {
this.acceptDocsOutOfOrder = acceptDocsOutOfOrder;
}
@Override
public void collect(int doc) throws IOException {}
@Override
public boolean acceptsDocsOutOfOrder() {
return acceptDocsOutOfOrder;
}
}
public void testBasic() throws Exception {
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);
acc.setScorer(new MockScorer());
@ -90,17 +79,12 @@ public class TestCachingCollector extends LuceneTestCase {
assertEquals(prevDocID + 1, doc);
prevDocID = doc;
}
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
});
}
}
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);
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());
try {
cc.replay(new NoOpCollector(false));
cc.replay(new NoOpCollector());
fail("replay should fail if CachingCollector is not cached");
} catch (IllegalStateException e) {
// 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 {
// tests the cached arrays allocation -- if the 'nextLength' was too high,
// 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;
for (boolean cacheScores : new boolean[] { false, true }) {
int bytesPerDoc = cacheScores ? 8 : 4;
CachingCollector cc = CachingCollector.create(new NoOpCollector(false),
CachingCollector cc = CachingCollector.create(new NoOpCollector(),
cacheScores, bytesPerDoc * ONE_BYTE * numDocs);
LeafCollector acc = cc.getLeafCollector(null);
acc.setScorer(new MockScorer());
@ -176,7 +133,7 @@ public class TestCachingCollector extends LuceneTestCase {
acc.collect(0);
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);
count[0]++;
}
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
});
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"));
query.add(sq1);
query.add(sq2);
TopScoreDocCollector collector = TopScoreDocCollector.create(1000, true);
TopScoreDocCollector collector = TopScoreDocCollector.create(1000);
searcher.search(query, collector);
hits = collector.topDocs().scoreDocs.length;
for (ScoreDoc scoreDoc : collector.topDocs().scoreDocs){

View File

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

View File

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

View File

@ -79,7 +79,7 @@ public class TestElevationComparator extends LuceneTestCase {
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);
TopDocs topDocs = topCollector.topDocs(0, 10);

View File

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

View File

@ -86,7 +86,7 @@ public class TestPositiveScoresOnlyCollector extends LuceneTestCase {
IndexSearcher searcher = newSearcher(ir);
Weight fake = new TermQuery(new Term("fake", "weight")).createWeight(searcher);
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);
LeafCollector ac = c.getLeafCollector(ir.leaves().get(0));
ac.setScorer(s);

View File

@ -91,10 +91,6 @@ public class TestScoreCachingWrappingScorer extends LuceneTestCase {
this.scorer = new ScoreCachingWrappingScorer(scorer);
}
@Override public boolean acceptsDocsOutOfOrder() {
return true;
}
}
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 {
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 {
base = context.docBase;
}
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
});
PhraseQuery pq = new PhraseQuery();
@ -133,10 +129,6 @@ public class TestSimilarity extends LuceneTestCase {
//System.out.println("Doc=" + doc + " score=" + score);
assertEquals(1.0f, scorer.score(), 0);
}
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
});
pq.setSlop(2);
@ -152,10 +144,6 @@ public class TestSimilarity extends LuceneTestCase {
//System.out.println("Doc=" + doc + " score=" + score);
assertEquals(2.0f, scorer.score(), 0);
}
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
});
reader.close();

View File

@ -191,11 +191,6 @@ public class TestSloppyPhraseQuery extends LuceneTestCase {
totalHits++;
max = Math.max(max, scorer.freq());
}
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
}
/** 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.score()));
}
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
});
QueryUtils.check(random(), pq, searcher);
}

View File

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

View File

@ -78,7 +78,7 @@ public class TestTermScorer extends LuceneTestCase {
Weight weight = indexSearcher.createNormalizedWeight(termQuery);
assertTrue(indexSearcher.getTopReaderContext() instanceof LeafReaderContext);
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
// other values
final List<TestHit> docs = new ArrayList<>();
@ -107,11 +107,6 @@ public class TestTermScorer extends LuceneTestCase {
protected void doSetNextReader(LeafReaderContext context) throws IOException {
base = context.docBase;
}
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
});
assertTrue("docs Size: " + docs.size() + " is not: " + 2, docs.size() == 2);
TestHit doc0 = docs.get(0);

View File

@ -357,11 +357,6 @@ public class TestTimeLimitingCollector extends LuceneTestCase {
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) {
// 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;
if (sort == null) {
if (useFrom) {
TopScoreDocCollector c = TopScoreDocCollector.create(numHits, random().nextBoolean());
TopScoreDocCollector c = TopScoreDocCollector.create(numHits);
searcher.search(query, c);
from = TestUtil.nextInt(random(), 0, numHits - 1);
size = numHits - from;
@ -217,7 +217,7 @@ public class TestTopDocsMerge extends LuceneTestCase {
topHits = searcher.search(query, numHits);
}
} 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);
if (useFrom) {
from = TestUtil.nextInt(random(), 0, numHits - 1);
@ -261,7 +261,7 @@ public class TestTopDocsMerge extends LuceneTestCase {
if (sort == null) {
subHits = subSearcher.search(w, numHits);
} 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);
subHits = c.topDocs(0, numHits);
}

View File

@ -63,7 +63,7 @@ public class TestTopFieldCollector extends LuceneTestCase {
for(int i = 0; i < sort.length; i++) {
Query q = new MatchAllDocsQuery();
TopDocsCollector<Entry> tdc = TopFieldCollector.create(sort[i], 10, false,
false, false, true);
false, false);
is.search(q, tdc);
@ -82,7 +82,7 @@ public class TestTopFieldCollector extends LuceneTestCase {
for(int i = 0; i < sort.length; i++) {
Query q = new MatchAllDocsQuery();
TopDocsCollector<Entry> tdc = TopFieldCollector.create(sort[i], 10, true, false,
false, true);
false);
is.search(q, tdc);
@ -102,7 +102,7 @@ public class TestTopFieldCollector extends LuceneTestCase {
for(int i = 0; i < sort.length; i++) {
Query q = new MatchAllDocsQuery();
TopDocsCollector<Entry> tdc = TopFieldCollector.create(sort[i], 10, true, true,
false, true);
false);
is.search(q, tdc);
@ -123,7 +123,7 @@ public class TestTopFieldCollector extends LuceneTestCase {
for(int i = 0; i < sort.length; i++) {
Query q = new MatchAllDocsQuery();
TopDocsCollector<Entry> tdc = TopFieldCollector.create(sort[i], 10, true, true,
false, true);
false);
is.search(q, tdc);
@ -143,7 +143,7 @@ public class TestTopFieldCollector extends LuceneTestCase {
for(int i = 0; i < sort.length; i++) {
Query q = new MatchAllDocsQuery();
TopDocsCollector<Entry> tdc = TopFieldCollector.create(sort[i], 10, true, true,
true, true);
true);
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 {
// Two Sort criteria to instantiate the multi/single comparators.
Sort[] sort = new Sort[] {new Sort(SortField.FIELD_DOC), new Sort() };
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();
assertEquals(0, td.totalHits);
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,
true,
doDocScores,
doMaxScore,
true);
doMaxScore);
DrillSidewaysResult r = search(query, hitCollector);
return new DrillSidewaysResult(r.facets, hitCollector.topDocs());
} else {
@ -227,7 +226,7 @@ public class DrillSideways {
limit = 1; // the collector does not alow numHits = 0
}
topN = Math.min(topN, limit);
TopScoreDocCollector hitCollector = TopScoreDocCollector.create(topN, after, true);
TopScoreDocCollector hitCollector = TopScoreDocCollector.create(topN, after);
DrillSidewaysResult r = search(query, hitCollector);
return new DrillSidewaysResult(r.facets, hitCollector.topDocs());
}
@ -236,12 +235,7 @@ public class DrillSideways {
* (e.g., {@code ToParentBlockJoinCollector}) expects all
* sub-scorers to be positioned on the document being
* collected. This will cause some performance loss;
* default is false. Note that if you return true from
* 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. */
* default is false. */
protected boolean scoreSubDocsAtOnce() {
return false;
}

View File

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

View File

@ -154,14 +154,6 @@ public class FacetsCollector extends SimpleCollector {
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
public final void collect(int doc) throws IOException {
docs.addDoc(doc);
@ -284,14 +276,9 @@ public class FacetsCollector extends SimpleCollector {
(FieldDoc) after,
fillFields,
doDocScores,
doMaxScore,
false);
doMaxScore);
} else {
// TODO: can we pass the right boolean for
// 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);
hitsCollector = TopScoreDocCollector.create(n, after);
}
searcher.search(q, MultiCollector.wrap(hitsCollector, fc));
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 {
lastDocID = -1;
}
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
});
// Also separately verify that DS respects the

View File

@ -126,11 +126,6 @@ public abstract class AbstractAllGroupHeadsCollector<GH extends AbstractAllGroup
groupHead.updateDocHead(doc);
}
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
/**
* Contains the result of group head retrieval.
* 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
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();
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
/**
* Returned by {@link AbstractDistinctValuesCollector#getGroups()},
* 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
protected void doSetNextReader(LeafReaderContext readerContext) throws IOException {
docBase = readerContext.docBase;

View File

@ -109,11 +109,6 @@ public abstract class AbstractGroupFacetCollector extends SimpleCollector {
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.
*/

View File

@ -69,10 +69,10 @@ public abstract class AbstractSecondPassGroupingCollector<GROUP_VALUE_TYPE> exte
final TopDocsCollector<?> collector;
if (withinGroupSort == null) {
// Sort by score
collector = TopScoreDocCollector.create(maxDocsPerGroup, true);
collector = TopScoreDocCollector.create(maxDocsPerGroup);
} else {
// Sort by fields
collector = TopFieldCollector.create(withinGroupSort, maxDocsPerGroup, fillSortFields, getScores, getMaxScores, true);
collector = TopFieldCollector.create(withinGroupSort, maxDocsPerGroup, fillSortFields, getScores, getMaxScores);
}
groupMap.put(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) {
@SuppressWarnings({"unchecked","rawtypes"})
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) {
throw new IllegalArgumentException("cannot sort by relevance within group: needsScores=false");
}
collector = TopScoreDocCollector.create(maxDocsPerGroup, true);
collector = TopScoreDocCollector.create(maxDocsPerGroup);
} else {
// 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);
@ -512,11 +512,6 @@ public class BlockGroupingCollector extends SimpleCollector {
}
}
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
@Override
protected void doSetNextReader(LeafReaderContext readerContext) throws IOException {
if (subDocUpto != 0) {

View File

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

View File

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

View File

@ -133,20 +133,24 @@ class TermsIncludingScoreQuery extends Query {
@Override
public Explanation explain(LeafReaderContext context, int doc) throws IOException {
SVInnerScorer scorer = (SVInnerScorer) bulkScorer(context, false, null);
if (scorer != null) {
return scorer.explain(doc);
Terms terms = context.reader().terms(field);
if (terms != null) {
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");
}
@Override
public boolean scoresDocsOutOfOrder() {
// We have optimized impls below if we are allowed
// to score out-of-order:
return true;
}
@Override
public Query getQuery() {
return TermsIncludingScoreQuery.this;
@ -179,143 +183,9 @@ class TermsIncludingScoreQuery extends Query {
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 {
final DocIdSetIterator matchingDocsIterator;

View File

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

View File

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

View File

@ -38,6 +38,9 @@ import java.util.*;
* IndexWriter#updateDocuments}). Ie, the join is computed
* at index time.
*
* <p>This collector MUST be used with {@link ToParentBlockJoinIndexSearcher},
* in order to work correctly.
*
* <p>The parent Sort must only use
* fields from the parent documents; sorting by field in
* 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) {
throw new IllegalArgumentException("cannot sort by relevance within group: trackScores=false");
}
collector = TopScoreDocCollector.create(numDocsInGroup, true);
collector = TopScoreDocCollector.create(numDocsInGroup);
} else {
// 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);

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");
}
@Override
public boolean scoresDocsOutOfOrder() {
return false;
}
}
static class BlockJoinScorer extends Scorer {

View File

@ -1314,6 +1314,7 @@ public class TestBlockJoin extends LuceneTestCase {
IndexReader r = w.getReader();
w.close();
IndexSearcher searcher = new ToParentBlockJoinIndexSearcher(r);
Query childQuery = new TermQuery(new Term("childText", "text"));
BitDocIdSetFilter parentsFilter = new BitDocIdSetCachingWrapperFilter(new QueryWrapperFilter(new TermQuery(new Term("isParent", "yes"))));
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)),
10, true, true);
newSearcher(r).search(parentQuery, c);
searcher.search(parentQuery, c);
TopGroups<Integer> groups = c.getTopGroups(childJoinQuery, null, 0, 10, 0, false);
// Two parents:
@ -1381,6 +1382,8 @@ public class TestBlockJoin extends LuceneTestCase {
IndexReader r = w.getReader();
w.close();
IndexSearcher searcher = new ToParentBlockJoinIndexSearcher(r);
// never matches:
Query childQuery = new TermQuery(new Term("childText", "bogus"));
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)),
10, true, true);
newSearcher(r).search(parentQuery, c);
searcher.search(parentQuery, c);
TopGroups<Integer> groups = c.getTopGroups(childJoinQuery, null, 0, 10, 0, false);
// 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);
}
}
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
});
indexSearcher.getIndexReader().close();
@ -447,8 +443,7 @@ public class TestJoinUtil extends LuceneTestCase {
dir,
newIndexWriterConfig(new MockAnalyzer(random(), MockTokenizer.KEYWORD, false)).setMergePolicy(newLogMergePolicy())
);
final boolean scoreDocsInOrder = TestJoinUtil.random().nextBoolean();
IndexIterationContext context = createContext(numberOfDocumentsToIndex, w, multipleValuesPerDocument, scoreDocsInOrder);
IndexIterationContext context = createContext(numberOfDocumentsToIndex, w, multipleValuesPerDocument);
IndexReader topLevelReader = w.getReader();
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...
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() {
@Override
@ -498,11 +493,6 @@ public class TestJoinUtil extends LuceneTestCase {
super.collect(doc);
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 {
return createContext(nDocs, writer, writer, multipleValuesPerDocument, scoreDocsInOrder);
private IndexIterationContext createContext(int nDocs, RandomIndexWriter writer, boolean multipleValuesPerDocument) throws IOException {
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();
int numRandomValues = nDocs / 2;
context.randomUniqueValues = new String[numRandomValues];
@ -683,11 +673,6 @@ public class TestJoinUtil extends LuceneTestCase {
public void setScorer(Scorer scorer) {
this.scorer = scorer;
}
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
});
} else {
fromSearcher.search(new TermQuery(new Term("value", uniqueRandomValue)), new SimpleCollector() {
@ -720,17 +705,11 @@ public class TestJoinUtil extends LuceneTestCase {
public void setScorer(Scorer scorer) {
this.scorer = scorer;
}
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
});
}
final Map<Integer, JoinScore> docToJoinScore = new HashMap<>();
if (multipleValuesPerDocument) {
if (scoreDocsInOrder) {
LeafReader slowCompositeReader = SlowCompositeReaderWrapper.wrap(toSearcher.getIndexReader());
Terms terms = slowCompositeReader.terms(toField);
if (terms != null) {
@ -757,43 +736,6 @@ public class TestJoinUtil extends LuceneTestCase {
} else {
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 int docBase;
@ -813,8 +755,6 @@ public class TestJoinUtil extends LuceneTestCase {
docBase = context.docBase;
}
@Override
public boolean acceptsDocsOutOfOrder() {return false;}
@Override
public void setScorer(Scorer scorer) {}
});

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -547,7 +547,7 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
//System.out.println("finalQuery=" + query);
// 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
// only retrieve the first num hits now:

View File

@ -18,21 +18,12 @@ package org.apache.lucene.search;
*/
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.WeakHashMap;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.util.VirtualMethod;
/** Wraps a Scorer with additional checks */
public 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);
final class AssertingBulkScorer extends BulkScorer {
public static BulkScorer wrap(Random random, BulkScorer other) {
if (other == null || other instanceof AssertingBulkScorer) {
@ -41,10 +32,6 @@ public class AssertingBulkScorer extends BulkScorer {
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 BulkScorer in;
@ -59,6 +46,7 @@ public class AssertingBulkScorer extends BulkScorer {
@Override
public void score(LeafCollector collector) throws IOException {
collector = new AssertingLeafCollector(random, collector, DocsEnum.NO_MORE_DOCS);
if (random.nextBoolean()) {
try {
final boolean remaining = in.score(collector, DocsEnum.NO_MORE_DOCS);
@ -73,6 +61,7 @@ public class AssertingBulkScorer extends BulkScorer {
@Override
public boolean score(LeafCollector collector, int max) throws IOException {
collector = new AssertingLeafCollector(random, 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.util.Random;
/** A crazy {@link BulkScorer} that wraps another {@link BulkScorer}
* but shuffles the order of the collected documents. */
public class AssertingBulkOutOfOrderScorer extends BulkScorer {
/** Wraps another Collector and checks that
* acceptsDocsOutOfOrder is respected. */
final BulkScorer in;
final Random random;
final class AssertingLeafCollector extends FilterLeafCollector {
public AssertingBulkOutOfOrderScorer(Random random, BulkScorer in) {
this.in = in;
private final Random random;
private final int max;
private Scorer scorer;
private int lastCollected = -1;
AssertingLeafCollector(Random random, LeafCollector collector, int max) {
super(collector);
this.random = random;
this.max = max;
}
@Override
public boolean score(LeafCollector collector, int max) throws IOException {
final RandomOrderCollector randomCollector = new RandomOrderCollector(random, collector);
final boolean remaining = in.score(randomCollector, max);
randomCollector.flush();
return remaining;
public void setScorer(Scorer scorer) throws IOException {
this.scorer = scorer;
super.setScorer(AssertingScorer.getAssertingScorer(random, scorer));
}
@Override
public void score(LeafCollector collector) throws IOException {
final RandomOrderCollector randomCollector = new RandomOrderCollector(random, collector);
in.score(randomCollector);
randomCollector.flush();
public void collect(int doc) throws IOException {
assert doc > lastCollected : "Out of order : " + lastCollected + " " + doc;
assert doc < max : "Out of range: " + doc + " >= " + max;
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);
}
final boolean scoresDocsOutOfOrder;
final Random random;
final Weight in;
AssertingWeight(Random random, Weight in) {
this.random = random;
this.in = in;
scoresDocsOutOfOrder = in.scoresDocsOutOfOrder() || random.nextBoolean();
}
@Override
@ -68,36 +66,12 @@ class AssertingWeight extends Weight {
}
@Override
public BulkScorer bulkScorer(LeafReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) throws IOException {
// if the caller asks for in-order scoring or if the weight does not support
// out-of order scoring then collection will have to happen in-order.
BulkScorer inScorer = in.bulkScorer(context, scoreDocsInOrder, acceptDocs);
public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
BulkScorer inScorer = in.bulkScorer(context, acceptDocs);
if (inScorer == null) {
return null;
}
if (AssertingBulkScorer.shouldWrap(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;
return AssertingBulkScorer.wrap(new Random(random.nextLong()), inScorer);
}
}

View File

@ -139,10 +139,6 @@ public class CheckHits {
protected void doSetNextReader(LeafReaderContext context) throws IOException {
base = context.docBase;
}
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
}
/**
@ -511,10 +507,6 @@ public class CheckHits {
protected void doSetNextReader(LeafReaderContext context) throws IOException {
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 {
//System.out.println("Checking "+q);
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 next_op = 1;
@ -323,11 +322,6 @@ public class QueryUtils {
this.scorer = null;
lastDoc[0] = -1;
}
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
});
if (lastReader[0] != null) {
@ -409,10 +403,6 @@ public class QueryUtils {
lastDoc[0] = -1;
liveDocs = context.reader().getLiveDocs();
}
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
});
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
}
@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
int group;
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);
}
@ -549,11 +549,6 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
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();
while (iterator.hasNext()) {
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);
}
@ -614,11 +609,6 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
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
protected void doSetNextReader(LeafReaderContext context) throws IOException {
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
protected void doSetNextReader(LeafReaderContext context) throws IOException {
this.contexts[context.ord] = context;

View File

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

View File

@ -87,9 +87,4 @@ public class DocSetCollector extends SimpleCollector {
protected void doSetNextReader(LeafReaderContext context) throws IOException {
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)) {
/**
* 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
public void collect(int doc) throws IOException {
super.collect(doc);

View File

@ -148,11 +148,6 @@ public class ExportQParserPlugin extends QParserPlugin {
++totalHits;
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 {
int groupDocsToCollect = getMax(groupOffset, docsPerGroup, maxDoc);
if (sort == null || sort == Sort.RELEVANCE) {
return TopScoreDocCollector.create(groupDocsToCollect, true);
return TopScoreDocCollector.create(groupDocsToCollect);
} 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;
Sort sort = cmd.getSort();
if(sort == null) {
this.mainCollector = TopScoreDocCollector.create(Math.max(this.reRankDocs, length),true);
this.mainCollector = TopScoreDocCollector.create(Math.max(this.reRankDocs, length));
} else {
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.reRankWeight = reRankWeight;

View File

@ -1597,7 +1597,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable,SolrIn
if (null == cmd.getSort()) {
assert null == cmd.getCursorMark() : "have cursor but no sort";
return TopScoreDocCollector.create(len, true);
return TopScoreDocCollector.create(len);
} else {
// we have a sort
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 FieldDoc searchAfter = (null != cursor ? cursor.getSearchAfterFieldDoc() : null);
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) {
numHits[0]++;
}
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
};
} else {
collector = new SimpleCollector() {
@ -1662,10 +1658,6 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable,SolrIn
float score = scorer.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();
if (score > topscore[0]) topscore[0] = score;
}
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
};
collector = MultiCollector.wrap(setCollector, topScoreCollector);

View File

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

View File

@ -785,11 +785,6 @@ public class TestRankQueryPlugin extends QParserPlugin {
public void collect(int doc) throws IOException {
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 trackMaxScores = 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<>();
// delegate and collect docs ourselves