LUCENE-8059: Fold early termination support into TopFieldCollector.

This commit is contained in:
Adrien Grand 2017-11-29 16:07:33 +01:00
parent d27ddcb409
commit 812711416d
23 changed files with 177 additions and 136 deletions

View File

@ -73,6 +73,10 @@ API Changes
* LUCENE-8049: IndexWriter.getMergingSegments()'s return type was changed from * LUCENE-8049: IndexWriter.getMergingSegments()'s return type was changed from
Collection to Set to more accurately reflect it's nature. (David Smiley) Collection to Set to more accurately reflect it's nature. (David Smiley)
* LUCENE-8059: TopFieldDocCollector can now early terminate collection when
the sort order is compatible with the index order. As a consequence,
EarlyTerminatingSortingCollector is now deprecated. (Adrien Grand)
New Features New Features
* LUCENE-8061: Add convenience factory methods to create BBoxes and XYZSolids * LUCENE-8061: Add convenience factory methods to create BBoxes and XYZSolids

View File

@ -113,7 +113,8 @@ public abstract class ReadTask extends PerfTask {
// pulling the Weight ourselves: // pulling the Weight ourselves:
TopFieldCollector collector = TopFieldCollector.create(sort, numHits, TopFieldCollector collector = TopFieldCollector.create(sort, numHits,
true, withScore(), true, withScore(),
withMaxScore()); withMaxScore(),
withTotalHits());
searcher.search(q, collector); searcher.search(q, collector);
hits = collector.topDocs(); hits = collector.topDocs();
} else { } else {
@ -220,6 +221,12 @@ public abstract class ReadTask extends PerfTask {
return true; return true;
} }
/** Whether totalHits should be computed (only useful with
* field sort) */
public boolean withTotalHits() {
return false;
}
/** /**
* Specify the number of hits to traverse. Tasks should override this if they want to restrict the number * Specify the number of hits to traverse. Tasks should override this if they want to restrict the number
* of hits that are traversed when {@link #withTraverse()} is true. Must be greater than 0. * of hits that are traversed when {@link #withTraverse()} is true. Must be greater than 0.

View File

@ -535,7 +535,8 @@ public class IndexSearcher {
@Override @Override
public TopFieldCollector newCollector() throws IOException { public TopFieldCollector newCollector() throws IOException {
final boolean fillFields = true; final boolean fillFields = true;
return TopFieldCollector.create(rewrittenSort, cappedNumHits, after, fillFields, doDocScores, doMaxScore); // TODO: don't pay the price for accurate hit counts by default
return TopFieldCollector.create(rewrittenSort, cappedNumHits, after, fillFields, doDocScores, doMaxScore, true);
} }
@Override @Override

View File

@ -54,7 +54,7 @@ public class SortRescorer extends Rescorer {
List<LeafReaderContext> leaves = searcher.getIndexReader().leaves(); List<LeafReaderContext> leaves = searcher.getIndexReader().leaves();
TopFieldCollector collector = TopFieldCollector.create(sort, topN, true, true, true); TopFieldCollector collector = TopFieldCollector.create(sort, topN, true, true, true, true);
// Now merge sort docIDs from hits, with reader's leaves: // Now merge sort docIDs from hits, with reader's leaves:
int hitUpto = 0; int hitUpto = 0;

View File

@ -18,6 +18,7 @@ package org.apache.lucene.search;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays;
import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.FieldValueHitQueue.Entry; import org.apache.lucene.search.FieldValueHitQueue.Entry;
@ -27,7 +28,7 @@ import org.apache.lucene.util.PriorityQueue;
* A {@link Collector} that sorts by {@link SortField} using * A {@link Collector} that sorts by {@link SortField} using
* {@link FieldComparator}s. * {@link FieldComparator}s.
* <p> * <p>
* See the {@link #create(org.apache.lucene.search.Sort, int, boolean, boolean, boolean)} method * See the {@link #create(org.apache.lucene.search.Sort, int, boolean, boolean, boolean, boolean)} method
* for instantiating a TopFieldCollector. * for instantiating a TopFieldCollector.
* *
* @lucene.experimental * @lucene.experimental
@ -67,20 +68,40 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
} }
} }
static boolean canEarlyTerminate(Sort searchSort, Sort indexSort) {
final SortField[] fields1 = searchSort.getSort();
final SortField[] fields2 = indexSort.getSort();
// early termination is possible if fields1 is a prefix of fields2
if (fields1.length > fields2.length) {
return false;
}
return Arrays.asList(fields1).equals(Arrays.asList(fields2).subList(0, fields1.length));
}
static int estimateRemainingHits(int hitCount, int doc, int maxDoc) {
double hitRatio = (double) hitCount / (doc + 1);
int remainingDocs = maxDoc - doc - 1;
int remainingHits = (int) (remainingDocs * hitRatio);
return remainingHits;
}
/* /*
* Implements a TopFieldCollector over one SortField criteria, with tracking * Implements a TopFieldCollector over one SortField criteria, with tracking
* document scores and maxScore. * document scores and maxScore.
*/ */
private static class SimpleFieldCollector extends TopFieldCollector { private static class SimpleFieldCollector extends TopFieldCollector {
final Sort sort;
final FieldValueHitQueue<Entry> queue; final FieldValueHitQueue<Entry> queue;
final boolean trackDocScores; final boolean trackDocScores;
final boolean trackMaxScore; final boolean trackMaxScore;
final boolean mayNeedScoresTwice; final boolean mayNeedScoresTwice;
final boolean trackTotalHits;
public SimpleFieldCollector(Sort sort, FieldValueHitQueue<Entry> queue, int numHits, boolean fillFields, public SimpleFieldCollector(Sort sort, FieldValueHitQueue<Entry> queue, int numHits, boolean fillFields,
boolean trackDocScores, boolean trackMaxScore) { boolean trackDocScores, boolean trackMaxScore, boolean trackTotalHits) {
super(queue, numHits, fillFields, sort.needsScores() || trackDocScores || trackMaxScore); super(queue, numHits, fillFields, sort.needsScores() || trackDocScores || trackMaxScore);
this.sort = sort;
this.queue = queue; this.queue = queue;
if (trackMaxScore) { if (trackMaxScore) {
maxScore = Float.NEGATIVE_INFINITY; // otherwise we would keep NaN maxScore = Float.NEGATIVE_INFINITY; // otherwise we would keep NaN
@ -91,6 +112,7 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
// we might call scorer.score() several times per doc so wrapping the scorer // we might call scorer.score() several times per doc so wrapping the scorer
// to cache scores would help // to cache scores would help
this.mayNeedScoresTwice = sort.needsScores() && (trackDocScores || trackMaxScore); this.mayNeedScoresTwice = sort.needsScores() && (trackDocScores || trackMaxScore);
this.trackTotalHits = trackTotalHits;
} }
@Override @Override
@ -99,6 +121,10 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
final LeafFieldComparator[] comparators = queue.getComparators(context); final LeafFieldComparator[] comparators = queue.getComparators(context);
final int[] reverseMul = queue.getReverseMul(); final int[] reverseMul = queue.getReverseMul();
final boolean canEarlyTerminate = trackTotalHits == false &&
trackMaxScore == false &&
canEarlyTerminate(sort, context.reader().getMetaData().getSort());
final int initialTotalHits = totalHits;
return new MultiComparatorLeafCollector(comparators, reverseMul, mayNeedScoresTwice) { return new MultiComparatorLeafCollector(comparators, reverseMul, mayNeedScoresTwice) {
@ -118,8 +144,17 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
// since docs are visited in doc Id order, if compare is 0, it means // since docs are visited in doc Id order, if compare is 0, it means
// this document is largest than anything else in the queue, and // this document is largest than anything else in the queue, and
// therefore not competitive. // therefore not competitive.
if (canEarlyTerminate) {
// scale totalHits linearly based on the number of docs
// and terminate collection
totalHits += estimateRemainingHits(totalHits - initialTotalHits, doc, context.reader().maxDoc());
earlyTerminated = true;
throw new CollectionTerminatedException();
} else {
// just move to the next doc
return; return;
} }
}
if (trackDocScores && !trackMaxScore) { if (trackDocScores && !trackMaxScore) {
score = scorer.score(); score = scorer.score();
@ -156,21 +191,25 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
*/ */
private final static class PagingFieldCollector extends TopFieldCollector { private final static class PagingFieldCollector extends TopFieldCollector {
final Sort sort;
int collectedHits; int collectedHits;
final FieldValueHitQueue<Entry> queue; final FieldValueHitQueue<Entry> queue;
final boolean trackDocScores; final boolean trackDocScores;
final boolean trackMaxScore; final boolean trackMaxScore;
final FieldDoc after; final FieldDoc after;
final boolean mayNeedScoresTwice; final boolean mayNeedScoresTwice;
final boolean trackTotalHits;
public PagingFieldCollector(Sort sort, FieldValueHitQueue<Entry> queue, FieldDoc after, int numHits, boolean fillFields, public PagingFieldCollector(Sort sort, FieldValueHitQueue<Entry> queue, FieldDoc after, int numHits, boolean fillFields,
boolean trackDocScores, boolean trackMaxScore) { boolean trackDocScores, boolean trackMaxScore, boolean trackTotalHits) {
super(queue, numHits, fillFields, trackDocScores || trackMaxScore || sort.needsScores()); super(queue, numHits, fillFields, trackDocScores || trackMaxScore || sort.needsScores());
this.sort = sort;
this.queue = queue; this.queue = queue;
this.trackDocScores = trackDocScores; this.trackDocScores = trackDocScores;
this.trackMaxScore = trackMaxScore; this.trackMaxScore = trackMaxScore;
this.after = after; this.after = after;
this.mayNeedScoresTwice = sort.needsScores() && (trackDocScores || trackMaxScore); this.mayNeedScoresTwice = sort.needsScores() && (trackDocScores || trackMaxScore);
this.trackTotalHits = trackTotalHits;
// Must set maxScore to NEG_INF, or otherwise Math.max always returns NaN. // Must set maxScore to NEG_INF, or otherwise Math.max always returns NaN.
maxScore = Float.NEGATIVE_INFINITY; maxScore = Float.NEGATIVE_INFINITY;
@ -188,6 +227,10 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException { public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
docBase = context.docBase; docBase = context.docBase;
final int afterDoc = after.doc - docBase; final int afterDoc = after.doc - docBase;
final boolean canEarlyTerminate = trackTotalHits == false &&
trackMaxScore == false &&
canEarlyTerminate(sort, context.reader().getMetaData().getSort());
final int initialTotalHits = totalHits;
return new MultiComparatorLeafCollector(queue.getComparators(context), queue.getReverseMul(), mayNeedScoresTwice) { return new MultiComparatorLeafCollector(queue.getComparators(context), queue.getReverseMul(), mayNeedScoresTwice) {
@Override @Override
@ -210,9 +253,18 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
final int cmp = reverseMul * comparator.compareBottom(doc); final int cmp = reverseMul * comparator.compareBottom(doc);
if (cmp <= 0) { if (cmp <= 0) {
// not competitive since documents are visited in doc id order // not competitive since documents are visited in doc id order
if (canEarlyTerminate) {
// scale totalHits linearly based on the number of docs
// and terminate collection
totalHits += estimateRemainingHits(totalHits - initialTotalHits, doc, context.reader().maxDoc());
earlyTerminated = true;
throw new CollectionTerminatedException();
} else {
// just move to the next doc
return; return;
} }
} }
}
final int topCmp = reverseMul * comparator.compareTop(doc); final int topCmp = reverseMul * comparator.compareTop(doc);
if (topCmp > 0 || (topCmp == 0 && doc <= afterDoc)) { if (topCmp > 0 || (topCmp == 0 && doc <= afterDoc)) {
@ -270,6 +322,7 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
FieldValueHitQueue.Entry bottom = null; FieldValueHitQueue.Entry bottom = null;
boolean queueFull; boolean queueFull;
int docBase; int docBase;
boolean earlyTerminated = false;
final boolean needsScores; final boolean needsScores;
// Declaring the constructor private prevents extending this class by anyone // Declaring the constructor private prevents extending this class by anyone
@ -318,12 +371,16 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
* true affects performance as it incurs the score computation on * true affects performance as it incurs the score computation on
* each result. Also, setting this true automatically sets * each result. Also, setting this true automatically sets
* <code>trackDocScores</code> to true as well. * <code>trackDocScores</code> to true as well.
* @param trackTotalHits
* specifies whether the total number of hits should be tracked. If
* set to false, the value of {@link TopFieldDocs#totalHits} will be
* approximated.
* @return a {@link TopFieldCollector} instance which will sort the results by * @return a {@link TopFieldCollector} instance which will sort the results by
* the sort criteria. * the sort criteria.
*/ */
public static TopFieldCollector create(Sort sort, int numHits, public static TopFieldCollector create(Sort sort, int numHits,
boolean fillFields, boolean trackDocScores, boolean trackMaxScore) { boolean fillFields, boolean trackDocScores, boolean trackMaxScore, boolean trackTotalHits) {
return create(sort, numHits, null, fillFields, trackDocScores, trackMaxScore); return create(sort, numHits, null, fillFields, trackDocScores, trackMaxScore, trackTotalHits);
} }
/** /**
@ -357,11 +414,15 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
* true affects performance as it incurs the score computation on * true affects performance as it incurs the score computation on
* each result. Also, setting this true automatically sets * each result. Also, setting this true automatically sets
* <code>trackDocScores</code> to true as well. * <code>trackDocScores</code> to true as well.
* @param trackTotalHits
* specifies whether the total number of hits should be tracked. If
* set to false, the value of {@link TopFieldDocs#totalHits} will be
* approximated.
* @return a {@link TopFieldCollector} instance which will sort the results by * @return a {@link TopFieldCollector} instance which will sort the results by
* the sort criteria. * the sort criteria.
*/ */
public static TopFieldCollector create(Sort sort, int numHits, FieldDoc after, public static TopFieldCollector create(Sort sort, int numHits, FieldDoc after,
boolean fillFields, boolean trackDocScores, boolean trackMaxScore) { boolean fillFields, boolean trackDocScores, boolean trackMaxScore, boolean trackTotalHits) {
if (sort.fields.length == 0) { if (sort.fields.length == 0) {
throw new IllegalArgumentException("Sort must contain at least one field"); throw new IllegalArgumentException("Sort must contain at least one field");
@ -374,7 +435,7 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
FieldValueHitQueue<Entry> queue = FieldValueHitQueue.create(sort.fields, numHits); FieldValueHitQueue<Entry> queue = FieldValueHitQueue.create(sort.fields, numHits);
if (after == null) { if (after == null) {
return new SimpleFieldCollector(sort, queue, numHits, fillFields, trackDocScores, trackMaxScore); return new SimpleFieldCollector(sort, queue, numHits, fillFields, trackDocScores, trackMaxScore, trackTotalHits);
} else { } else {
if (after.fields == null) { if (after.fields == null) {
throw new IllegalArgumentException("after.fields wasn't set; you must pass fillFields=true for the previous search"); throw new IllegalArgumentException("after.fields wasn't set; you must pass fillFields=true for the previous search");
@ -384,7 +445,7 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
throw new IllegalArgumentException("after.fields has " + after.fields.length + " values but sort has " + sort.getSort().length); throw new IllegalArgumentException("after.fields has " + after.fields.length + " values but sort has " + sort.getSort().length);
} }
return new PagingFieldCollector(sort, queue, after, numHits, fillFields, trackDocScores, trackMaxScore); return new PagingFieldCollector(sort, queue, after, numHits, fillFields, trackDocScores, trackMaxScore, trackTotalHits);
} }
} }
@ -442,4 +503,9 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
public TopFieldDocs topDocs() { public TopFieldDocs topDocs() {
return (TopFieldDocs) super.topDocs(); return (TopFieldDocs) super.topDocs();
} }
/** Return whether collection terminated early. */
public boolean isEarlyTerminated() {
return earlyTerminated;
}
} }

View File

@ -61,7 +61,6 @@ import org.apache.lucene.document.StoredField;
import org.apache.lucene.document.StringField; import org.apache.lucene.document.StringField;
import org.apache.lucene.document.TextField; import org.apache.lucene.document.TextField;
import org.apache.lucene.search.CollectionStatistics; import org.apache.lucene.search.CollectionStatistics;
import org.apache.lucene.search.EarlyTerminatingSortingCollector;
import org.apache.lucene.search.FieldDoc; import org.apache.lucene.search.FieldDoc;
import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.MatchAllDocsQuery;
@ -2335,13 +2334,12 @@ public class TestIndexSorting extends LuceneTestCase {
System.out.println("TEST: iter=" + iter + " numHits=" + numHits); System.out.println("TEST: iter=" + iter + " numHits=" + numHits);
} }
TopFieldCollector c1 = TopFieldCollector.create(sort, numHits, true, true, true); TopFieldCollector c1 = TopFieldCollector.create(sort, numHits, true, true, true, true);
s1.search(new MatchAllDocsQuery(), c1); s1.search(new MatchAllDocsQuery(), c1);
TopDocs hits1 = c1.topDocs(); TopDocs hits1 = c1.topDocs();
TopFieldCollector c2 = TopFieldCollector.create(sort, numHits, true, true, true); TopFieldCollector c2 = TopFieldCollector.create(sort, numHits, true, true, true, false);
EarlyTerminatingSortingCollector c3 = new EarlyTerminatingSortingCollector(c2, sort, numHits); s2.search(new MatchAllDocsQuery(), c2);
s2.search(new MatchAllDocsQuery(), c3);
TopDocs hits2 = c2.topDocs(); TopDocs hits2 = c2.topDocs();
@ -2362,7 +2360,7 @@ public class TestIndexSorting extends LuceneTestCase {
ScoreDoc hit1 = hits1.scoreDocs[i]; ScoreDoc hit1 = hits1.scoreDocs[i];
ScoreDoc hit2 = hits2.scoreDocs[i]; ScoreDoc hit2 = hits2.scoreDocs[i];
assertEquals(r1.document(hit1.doc).get("id"), r2.document(hit2.doc).get("id")); assertEquals(r1.document(hit1.doc).get("id"), r2.document(hit2.doc).get("id"));
assertEquals(((FieldDoc) hit1).fields, ((FieldDoc) hit2).fields); assertArrayEquals(((FieldDoc) hit1).fields, ((FieldDoc) hit2).fields);
} }
} }

View File

@ -386,10 +386,10 @@ public class TestBoolean2 extends LuceneTestCase {
} }
// check diff (randomized) scorers (from AssertingSearcher) produce the same results // check diff (randomized) scorers (from AssertingSearcher) produce the same results
TopFieldCollector collector = TopFieldCollector.create(sort, 1000, false, true, true); TopFieldCollector collector = TopFieldCollector.create(sort, 1000, false, true, true, false);
searcher.search(q1, collector); searcher.search(q1, collector);
ScoreDoc[] hits1 = collector.topDocs().scoreDocs; ScoreDoc[] hits1 = collector.topDocs().scoreDocs;
collector = TopFieldCollector.create(sort, 1000, false, true, true); collector = TopFieldCollector.create(sort, 1000, false, true, true, false);
searcher.search(q1, collector); searcher.search(q1, collector);
ScoreDoc[] hits2 = collector.topDocs().scoreDocs; ScoreDoc[] hits2 = collector.topDocs().scoreDocs;
tot+=hits2.length; tot+=hits2.length;
@ -402,10 +402,10 @@ public class TestBoolean2 extends LuceneTestCase {
assertEquals(mulFactor*collector.totalHits + NUM_EXTRA_DOCS/2, hits4.totalHits); assertEquals(mulFactor*collector.totalHits + NUM_EXTRA_DOCS/2, hits4.totalHits);
// test diff (randomized) scorers produce the same results on bigSearcher as well // test diff (randomized) scorers produce the same results on bigSearcher as well
collector = TopFieldCollector.create(sort, 1000 * mulFactor, false, true, true); collector = TopFieldCollector.create(sort, 1000 * mulFactor, false, true, true, false);
bigSearcher.search(q1, collector); bigSearcher.search(q1, collector);
hits1 = collector.topDocs().scoreDocs; hits1 = collector.topDocs().scoreDocs;
collector = TopFieldCollector.create(sort, 1000 * mulFactor, false, true, true); collector = TopFieldCollector.create(sort, 1000 * mulFactor, false, true, true, false);
bigSearcher.search(q1, collector); bigSearcher.search(q1, collector);
hits2 = collector.topDocs().scoreDocs; hits2 = collector.topDocs().scoreDocs;
CheckHits.checkEqual(q1, hits1, hits2); CheckHits.checkEqual(q1, hits1, hits2);

View File

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

View File

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

View File

@ -71,7 +71,7 @@ public class TestTopFieldCollector extends LuceneTestCase {
for(int i = 0; i < sort.length; i++) { for(int i = 0; i < sort.length; i++) {
Query q = new MatchAllDocsQuery(); Query q = new MatchAllDocsQuery();
TopDocsCollector<Entry> tdc = TopFieldCollector.create(sort[i], 10, false, TopDocsCollector<Entry> tdc = TopFieldCollector.create(sort[i], 10, false,
false, false); false, false, true);
is.search(q, tdc); is.search(q, tdc);
@ -90,7 +90,7 @@ public class TestTopFieldCollector extends LuceneTestCase {
for(int i = 0; i < sort.length; i++) { for(int i = 0; i < sort.length; i++) {
Query q = new MatchAllDocsQuery(); Query q = new MatchAllDocsQuery();
TopDocsCollector<Entry> tdc = TopFieldCollector.create(sort[i], 10, true, false, TopDocsCollector<Entry> tdc = TopFieldCollector.create(sort[i], 10, true, false,
false); false, true);
is.search(q, tdc); is.search(q, tdc);
@ -110,7 +110,7 @@ public class TestTopFieldCollector extends LuceneTestCase {
for(int i = 0; i < sort.length; i++) { for(int i = 0; i < sort.length; i++) {
Query q = new MatchAllDocsQuery(); Query q = new MatchAllDocsQuery();
TopDocsCollector<Entry> tdc = TopFieldCollector.create(sort[i], 10, true, true, TopDocsCollector<Entry> tdc = TopFieldCollector.create(sort[i], 10, true, true,
false); false, true);
is.search(q, tdc); is.search(q, tdc);
@ -131,7 +131,7 @@ public class TestTopFieldCollector extends LuceneTestCase {
for(int i = 0; i < sort.length; i++) { for(int i = 0; i < sort.length; i++) {
Query q = new MatchAllDocsQuery(); Query q = new MatchAllDocsQuery();
TopDocsCollector<Entry> tdc = TopFieldCollector.create(sort[i], 10, true, true, TopDocsCollector<Entry> tdc = TopFieldCollector.create(sort[i], 10, true, true,
false); false, true);
is.search(q, tdc); is.search(q, tdc);
@ -151,7 +151,7 @@ public class TestTopFieldCollector extends LuceneTestCase {
for(int i = 0; i < sort.length; i++) { for(int i = 0; i < sort.length; i++) {
Query q = new MatchAllDocsQuery(); Query q = new MatchAllDocsQuery();
TopDocsCollector<Entry> tdc = TopFieldCollector.create(sort[i], 10, true, true, TopDocsCollector<Entry> tdc = TopFieldCollector.create(sort[i], 10, true, true,
true); true, true);
is.search(q, tdc); is.search(q, tdc);
@ -169,7 +169,7 @@ public class TestTopFieldCollector extends LuceneTestCase {
// Two Sort criteria to instantiate the multi/single comparators. // Two Sort criteria to instantiate the multi/single comparators.
Sort[] sort = new Sort[] {new Sort(SortField.FIELD_DOC), new Sort() }; Sort[] sort = new Sort[] {new Sort(SortField.FIELD_DOC), new Sort() };
for(int i = 0; i < sort.length; i++) { for(int i = 0; i < sort.length; i++) {
TopDocsCollector<Entry> tdc = TopFieldCollector.create(sort[i], 10, true, true, true); TopDocsCollector<Entry> tdc = TopFieldCollector.create(sort[i], 10, true, true, true, true);
TopDocs td = tdc.topDocs(); TopDocs td = tdc.topDocs();
assertEquals(0, td.totalHits); assertEquals(0, td.totalHits);
assertTrue(Float.isNaN(td.getMaxScore())); assertTrue(Float.isNaN(td.getMaxScore()));
@ -204,7 +204,7 @@ public class TestTopFieldCollector extends LuceneTestCase {
for (Sort sort : new Sort[] {new Sort(SortField.FIELD_SCORE), new Sort(new SortField("f", SortField.Type.SCORE))}) { for (Sort sort : new Sort[] {new Sort(SortField.FIELD_SCORE), new Sort(new SortField("f", SortField.Type.SCORE))}) {
for (boolean doDocScores : new boolean[] {false, true}) { for (boolean doDocScores : new boolean[] {false, true}) {
for (boolean doMaxScore : new boolean[] {false, true}) { for (boolean doMaxScore : new boolean[] {false, true}) {
final TopFieldCollector topCollector = TopFieldCollector.create(sort, TestUtil.nextInt(random(), 1, 2), true, doDocScores, doMaxScore); final TopFieldCollector topCollector = TopFieldCollector.create(sort, TestUtil.nextInt(random(), 1, 2), true, doDocScores, doMaxScore, true);
final Collector assertingCollector = new Collector() { final Collector assertingCollector = new Collector() {
@Override @Override
public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException { public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {

View File

@ -30,8 +30,8 @@ import org.apache.lucene.document.NumericDocValuesField;
import org.apache.lucene.document.StringField; import org.apache.lucene.document.StringField;
import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriterConfig; import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.MockRandomMergePolicy; import org.apache.lucene.index.MockRandomMergePolicy;
import org.apache.lucene.index.QueryTimeout;
import org.apache.lucene.index.RandomIndexWriter; import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.SerialMergeScheduler; import org.apache.lucene.index.SerialMergeScheduler;
import org.apache.lucene.index.Term; import org.apache.lucene.index.Term;
@ -41,7 +41,7 @@ import org.apache.lucene.util.TestUtil;
import com.carrotsearch.randomizedtesting.generators.RandomPicks; import com.carrotsearch.randomizedtesting.generators.RandomPicks;
public class TestEarlyTerminatingSortingCollector extends LuceneTestCase { public class TestTopFieldCollectorEarlyTermination extends LuceneTestCase {
private int numDocs; private int numDocs;
private List<String> terms; private List<String> terms;
@ -111,18 +111,36 @@ public class TestEarlyTerminatingSortingCollector extends LuceneTestCase {
} }
public void testEarlyTermination() throws IOException { public void testEarlyTermination() throws IOException {
doTestEarlyTermination(false);
}
public void testEarlyTerminationWhenPaging() throws IOException {
doTestEarlyTermination(true);
}
private void doTestEarlyTermination(boolean paging) throws IOException {
final int iters = atLeast(8); final int iters = atLeast(8);
for (int i = 0; i < iters; ++i) { for (int i = 0; i < iters; ++i) {
createRandomIndex(false); createRandomIndex(false);
int maxSegmentSize = 0;
for (LeafReaderContext ctx : reader.leaves()) {
maxSegmentSize = Math.max(ctx.reader().numDocs(), maxSegmentSize);
}
for (int j = 0; j < iters; ++j) { for (int j = 0; j < iters; ++j) {
final IndexSearcher searcher = newSearcher(reader); final IndexSearcher searcher = newSearcher(reader);
final int numHits = TestUtil.nextInt(random(), 1, numDocs); final int numHits = TestUtil.nextInt(random(), 1, numDocs);
final Sort sort = new Sort(new SortField("ndv1", SortField.Type.LONG, false)); FieldDoc after;
if (paging) {
TopFieldDocs td = searcher.search(new MatchAllDocsQuery(), 10, sort);
after = (FieldDoc) td.scoreDocs[td.scoreDocs.length - 1];
} else {
after = null;
}
final boolean fillFields = random().nextBoolean(); final boolean fillFields = random().nextBoolean();
final boolean trackDocScores = random().nextBoolean(); final boolean trackDocScores = random().nextBoolean();
final boolean trackMaxScore = random().nextBoolean(); final boolean trackMaxScore = random().nextBoolean();
final TopFieldCollector collector1 = TopFieldCollector.create(sort, numHits, fillFields, trackDocScores, trackMaxScore); final TopFieldCollector collector1 = TopFieldCollector.create(sort, numHits, after, fillFields, trackDocScores, trackMaxScore, true);
final TopFieldCollector collector2 = TopFieldCollector.create(sort, numHits, fillFields, trackDocScores, trackMaxScore); final TopFieldCollector collector2 = TopFieldCollector.create(sort, numHits, after, fillFields, trackDocScores, trackMaxScore, false);
final Query query; final Query query;
if (random().nextBoolean()) { if (random().nextBoolean()) {
@ -131,115 +149,64 @@ public class TestEarlyTerminatingSortingCollector extends LuceneTestCase {
query = new MatchAllDocsQuery(); query = new MatchAllDocsQuery();
} }
searcher.search(query, collector1); searcher.search(query, collector1);
searcher.search(query, new EarlyTerminatingSortingCollector(collector2, sort, numHits)); searcher.search(query, collector2);
assertTrue(collector1.getTotalHits() >= collector2.getTotalHits()); TopDocs td1 = collector1.topDocs();
assertTopDocsEquals(collector1.topDocs().scoreDocs, collector2.topDocs().scoreDocs); TopDocs td2 = collector2.topDocs();
assertFalse(collector1.isEarlyTerminated());
if (trackMaxScore == false && paging == false && maxSegmentSize >= numHits && query instanceof MatchAllDocsQuery) {
// Make sure that we sometimes early terminate
assertTrue(collector2.isEarlyTerminated());
}
if (collector2.isEarlyTerminated()) {
assertTrue(td2.totalHits >= td1.scoreDocs.length);
assertTrue(td2.totalHits <= reader.maxDoc());
} else {
assertEquals(td2.totalHits, td1.totalHits);
}
assertTopDocsEquals(td1.scoreDocs, td2.scoreDocs);
} }
closeIndex(); closeIndex();
} }
} }
public void testCanEarlyTerminate() { public void testCanEarlyTerminate() {
assertTrue(EarlyTerminatingSortingCollector.canEarlyTerminate( assertTrue(TopFieldCollector.canEarlyTerminate(
new Sort(new SortField("a", SortField.Type.LONG)), new Sort(new SortField("a", SortField.Type.LONG)),
new Sort(new SortField("a", SortField.Type.LONG)))); new Sort(new SortField("a", SortField.Type.LONG))));
assertTrue(EarlyTerminatingSortingCollector.canEarlyTerminate( assertTrue(TopFieldCollector.canEarlyTerminate(
new Sort(new SortField("a", SortField.Type.LONG), new SortField("b", SortField.Type.STRING)), new Sort(new SortField("a", SortField.Type.LONG), new SortField("b", SortField.Type.STRING)),
new Sort(new SortField("a", SortField.Type.LONG), new SortField("b", SortField.Type.STRING)))); new Sort(new SortField("a", SortField.Type.LONG), new SortField("b", SortField.Type.STRING))));
assertTrue(EarlyTerminatingSortingCollector.canEarlyTerminate( assertTrue(TopFieldCollector.canEarlyTerminate(
new Sort(new SortField("a", SortField.Type.LONG)), new Sort(new SortField("a", SortField.Type.LONG)),
new Sort(new SortField("a", SortField.Type.LONG), new SortField("b", SortField.Type.STRING)))); new Sort(new SortField("a", SortField.Type.LONG), new SortField("b", SortField.Type.STRING))));
assertFalse(EarlyTerminatingSortingCollector.canEarlyTerminate( assertFalse(TopFieldCollector.canEarlyTerminate(
new Sort(new SortField("a", SortField.Type.LONG, true)), new Sort(new SortField("a", SortField.Type.LONG, true)),
new Sort(new SortField("a", SortField.Type.LONG, false)))); new Sort(new SortField("a", SortField.Type.LONG, false))));
assertFalse(EarlyTerminatingSortingCollector.canEarlyTerminate( assertFalse(TopFieldCollector.canEarlyTerminate(
new Sort(new SortField("a", SortField.Type.LONG), new SortField("b", SortField.Type.STRING)), new Sort(new SortField("a", SortField.Type.LONG), new SortField("b", SortField.Type.STRING)),
new Sort(new SortField("a", SortField.Type.LONG)))); new Sort(new SortField("a", SortField.Type.LONG))));
assertFalse(EarlyTerminatingSortingCollector.canEarlyTerminate( assertFalse(TopFieldCollector.canEarlyTerminate(
new Sort(new SortField("a", SortField.Type.LONG), new SortField("b", SortField.Type.STRING)), new Sort(new SortField("a", SortField.Type.LONG), new SortField("b", SortField.Type.STRING)),
new Sort(new SortField("a", SortField.Type.LONG), new SortField("c", SortField.Type.STRING)))); new Sort(new SortField("a", SortField.Type.LONG), new SortField("c", SortField.Type.STRING))));
assertFalse(EarlyTerminatingSortingCollector.canEarlyTerminate( assertFalse(TopFieldCollector.canEarlyTerminate(
new Sort(new SortField("a", SortField.Type.LONG), new SortField("b", SortField.Type.STRING)), new Sort(new SortField("a", SortField.Type.LONG), new SortField("b", SortField.Type.STRING)),
new Sort(new SortField("c", SortField.Type.LONG), new SortField("b", SortField.Type.STRING)))); new Sort(new SortField("c", SortField.Type.LONG), new SortField("b", SortField.Type.STRING))));
} }
public void testEarlyTerminationDifferentSorter() throws IOException {
createRandomIndex(true);
Sort sort = new Sort(new SortField("ndv2", SortField.Type.LONG, false));
Collector c = new EarlyTerminatingSortingCollector(TopFieldCollector.create(sort, 10, true, true, true), sort, 10);
IndexSearcher searcher = newSearcher(reader);
Exception e = expectThrows(IllegalStateException.class,
() -> {
searcher.search(new MatchAllDocsQuery(), c);
});
assertEquals("Cannot early terminate with sort order <long: \"ndv2\"> if segments are sorted with <long: \"ndv1\">", e.getMessage());
closeIndex();
}
private static void assertTopDocsEquals(ScoreDoc[] scoreDocs1, ScoreDoc[] scoreDocs2) { private static void assertTopDocsEquals(ScoreDoc[] scoreDocs1, ScoreDoc[] scoreDocs2) {
assertEquals(scoreDocs1.length, scoreDocs2.length); assertEquals(scoreDocs1.length, scoreDocs2.length);
for (int i = 0; i < scoreDocs1.length; ++i) { for (int i = 0; i < scoreDocs1.length; ++i) {
final ScoreDoc scoreDoc1 = scoreDocs1[i]; final ScoreDoc scoreDoc1 = scoreDocs1[i];
final ScoreDoc scoreDoc2 = scoreDocs2[i]; final ScoreDoc scoreDoc2 = scoreDocs2[i];
assertEquals(scoreDoc1.doc, scoreDoc2.doc); assertEquals(scoreDoc1.doc, scoreDoc2.doc);
assertEquals(scoreDoc1.score, scoreDoc2.score, 0.001f); assertEquals(scoreDoc1.score, scoreDoc2.score, 0f);
}
}
private static class TestTerminatedEarlySimpleCollector extends SimpleCollector {
private boolean collectedSomething;
public boolean collectedSomething() {
return collectedSomething;
}
@Override
public void collect(int doc) throws IOException {
collectedSomething = true;
}
@Override
public boolean needsScores() {
return false;
}
}
private static class TestEarlyTerminatingSortingcollectorQueryTimeout implements QueryTimeout {
final private boolean shouldExit;
public TestEarlyTerminatingSortingcollectorQueryTimeout(boolean shouldExit) {
this.shouldExit = shouldExit;
}
public boolean shouldExit() {
return shouldExit;
}
}
public void testTerminatedEarly() throws IOException {
final int iters = atLeast(8);
for (int i = 0; i < iters; ++i) {
createRandomIndex(true);
final IndexSearcher searcher = new IndexSearcher(reader); // future TODO: use newSearcher(reader);
final Query query = new MatchAllDocsQuery(); // search for everything/anything
final TestTerminatedEarlySimpleCollector collector1 = new TestTerminatedEarlySimpleCollector();
searcher.search(query, collector1);
final TestTerminatedEarlySimpleCollector collector2 = new TestTerminatedEarlySimpleCollector();
final EarlyTerminatingSortingCollector etsCollector = new EarlyTerminatingSortingCollector(collector2, sort, 1);
searcher.search(query, etsCollector);
assertTrue("collector1="+collector1.collectedSomething()+" vs. collector2="+collector2.collectedSomething(), collector1.collectedSomething() == collector2.collectedSomething());
if (collector1.collectedSomething()) {
// we collected something and since we modestly asked for just one document we should have terminated early
assertTrue("should have terminated early (searcher.reader="+searcher.reader+")", etsCollector.terminatedEarly());
}
closeIndex();
} }
} }

View File

@ -240,7 +240,7 @@ public class DrillSideways {
@Override @Override
public TopFieldCollector newCollector() throws IOException { public TopFieldCollector newCollector() throws IOException {
return TopFieldCollector.create(sort, fTopN, after, true, doDocScores, doMaxScore); return TopFieldCollector.create(sort, fTopN, after, true, doDocScores, doMaxScore, true);
} }
@Override @Override
@ -259,7 +259,7 @@ public class DrillSideways {
} else { } else {
final TopFieldCollector hitCollector = final TopFieldCollector hitCollector =
TopFieldCollector.create(sort, fTopN, after, true, doDocScores, doMaxScore); TopFieldCollector.create(sort, fTopN, after, true, doDocScores, doMaxScore, true);
DrillSidewaysResult r = search(query, hitCollector); DrillSidewaysResult r = search(query, hitCollector);
return new DrillSidewaysResult(r.facets, hitCollector.topDocs()); return new DrillSidewaysResult(r.facets, hitCollector.topDocs());
} }

View File

@ -232,7 +232,8 @@ public class FacetsCollector extends SimpleCollector implements Collector {
(FieldDoc) after, (FieldDoc) after,
fillFields, fillFields,
doDocScores, doDocScores,
doMaxScore); doMaxScore,
true); // TODO: can we disable exact hit counts
} else { } else {
hitsCollector = TopScoreDocCollector.create(n, after); hitsCollector = TopScoreDocCollector.create(n, after);
} }

View File

@ -305,7 +305,7 @@ public class BlockGroupingCollector extends SimpleCollector {
collector = TopScoreDocCollector.create(maxDocsPerGroup); collector = TopScoreDocCollector.create(maxDocsPerGroup);
} else { } else {
// Sort by fields // Sort by fields
collector = TopFieldCollector.create(withinGroupSort, maxDocsPerGroup, fillSortFields, needsScores, needsScores); collector = TopFieldCollector.create(withinGroupSort, maxDocsPerGroup, fillSortFields, needsScores, needsScores, true); // TODO: disable exact counts?
} }
LeafCollector leafCollector = collector.getLeafCollector(og.readerContext); LeafCollector leafCollector = collector.getLeafCollector(og.readerContext);

View File

@ -70,7 +70,7 @@ public class TopGroupsCollector<T> extends SecondPassGroupingCollector<T> {
this.needsScores = getScores || getMaxScores || withinGroupSort.needsScores(); this.needsScores = getScores || getMaxScores || withinGroupSort.needsScores();
this.supplier = withinGroupSort == Sort.RELEVANCE ? this.supplier = withinGroupSort == Sort.RELEVANCE ?
() -> TopScoreDocCollector.create(maxDocsPerGroup) : () -> TopScoreDocCollector.create(maxDocsPerGroup) :
() -> TopFieldCollector.create(withinGroupSort, maxDocsPerGroup, fillSortFields, getScores, getMaxScores); () -> TopFieldCollector.create(withinGroupSort, maxDocsPerGroup, fillSortFields, getScores, getMaxScores, true); // TODO: disable exact counts?
} }
@Override @Override

View File

@ -59,8 +59,6 @@ import org.apache.lucene.index.Term;
import org.apache.lucene.search.BooleanClause.Occur; import org.apache.lucene.search.BooleanClause.Occur;
import org.apache.lucene.search.BooleanClause; import org.apache.lucene.search.BooleanClause;
import org.apache.lucene.search.BooleanQuery; import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.Collector;
import org.apache.lucene.search.EarlyTerminatingSortingCollector;
import org.apache.lucene.search.FieldDoc; import org.apache.lucene.search.FieldDoc;
import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.PrefixQuery; import org.apache.lucene.search.PrefixQuery;
@ -649,11 +647,7 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
//System.out.println("finalQuery=" + finalQuery); //System.out.println("finalQuery=" + finalQuery);
// Sort by weight, descending: // Sort by weight, descending:
TopFieldCollector c = TopFieldCollector.create(SORT, num, true, false, false); TopFieldCollector c = TopFieldCollector.create(SORT, num, true, false, false, false);
// We sorted postings by weight during indexing, so we
// only retrieve the first num hits now:
Collector c2 = new EarlyTerminatingSortingCollector(c, SORT, num);
List<LookupResult> results = null; List<LookupResult> results = null;
SearcherManager mgr; SearcherManager mgr;
IndexSearcher searcher; IndexSearcher searcher;
@ -663,7 +657,7 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
} }
try { try {
//System.out.println("got searcher=" + searcher); //System.out.println("got searcher=" + searcher);
searcher.search(finalQuery, c2); searcher.search(finalQuery, c);
TopFieldDocs hits = c.topDocs(); TopFieldDocs hits = c.topDocs();

View File

@ -537,7 +537,7 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
DocIdSetIterator iterator = new BitSetIterator(groupBits, 0); // cost is not useful here DocIdSetIterator iterator = new BitSetIterator(groupBits, 0); // cost is not useful here
int group; int group;
while ((group = iterator.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) { while ((group = iterator.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
Collector collector = (sort == null) ? TopScoreDocCollector.create(limit) : TopFieldCollector.create(sort, limit, false, false, false); Collector collector = (sort == null) ? TopScoreDocCollector.create(limit) : TopFieldCollector.create(sort, limit, false, false, false, true);
groups.put(group, collector); groups.put(group, collector);
} }
@ -629,7 +629,7 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
Iterator<LongCursor> iterator = groupSet.iterator(); Iterator<LongCursor> iterator = groupSet.iterator();
while (iterator.hasNext()) { while (iterator.hasNext()) {
LongCursor cursor = iterator.next(); LongCursor cursor = iterator.next();
Collector collector = (sort == null) ? TopScoreDocCollector.create(limit) : TopFieldCollector.create(sort, limit, false, false, false); Collector collector = (sort == null) ? TopScoreDocCollector.create(limit) : TopFieldCollector.create(sort, limit, false, false, false, true);
groups.put(cursor.value, collector); groups.put(cursor.value, collector);
} }

View File

@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.lucene.search; package org.apache.solr.search;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;
@ -28,7 +28,9 @@ import org.apache.lucene.search.FilterCollector;
import org.apache.lucene.search.FilterLeafCollector; import org.apache.lucene.search.FilterLeafCollector;
import org.apache.lucene.search.LeafCollector; import org.apache.lucene.search.LeafCollector;
import org.apache.lucene.search.Sort; import org.apache.lucene.search.Sort;
import org.apache.lucene.search.SortField;
import org.apache.lucene.search.TopDocsCollector; import org.apache.lucene.search.TopDocsCollector;
import org.apache.lucene.search.TopFieldCollector;
import org.apache.lucene.search.TotalHitCountCollector; import org.apache.lucene.search.TotalHitCountCollector;
/** /**
@ -49,10 +51,11 @@ import org.apache.lucene.search.TotalHitCountCollector;
* hit count} will be vastly underestimated since not all matching documents will have * hit count} will be vastly underestimated since not all matching documents will have
* been collected. * been collected.
* *
* @deprecated Use {@link TopFieldCollector} and set trackTotalHits to false.
* @lucene.experimental * @lucene.experimental
*/ */
@Deprecated
public class EarlyTerminatingSortingCollector extends FilterCollector { final class EarlyTerminatingSortingCollector extends FilterCollector {
/** Returns whether collection can be early-terminated if it sorts with the /** Returns whether collection can be early-terminated if it sorts with the
* provided {@link Sort} and if segments are merged with the provided * provided {@link Sort} and if segments are merged with the provided

View File

@ -881,7 +881,7 @@ public class Grouping {
if (sort == null || sort.equals(Sort.RELEVANCE)) { if (sort == null || sort.equals(Sort.RELEVANCE)) {
return TopScoreDocCollector.create(groupDocsToCollect); return TopScoreDocCollector.create(groupDocsToCollect);
} else { } else {
return TopFieldCollector.create(searcher.weightSort(sort), groupDocsToCollect, false, needScores, needScores); return TopFieldCollector.create(searcher.weightSort(sort), groupDocsToCollect, false, needScores, needScores, true);
} }
} }

View File

@ -64,7 +64,7 @@ public class ReRankCollector extends TopDocsCollector {
this.mainCollector = TopScoreDocCollector.create(Math.max(this.reRankDocs, length)); this.mainCollector = TopScoreDocCollector.create(Math.max(this.reRankDocs, length));
} else { } else {
sort = sort.rewrite(searcher); sort = sort.rewrite(searcher);
this.mainCollector = TopFieldCollector.create(sort, Math.max(this.reRankDocs, length), false, true, true); this.mainCollector = TopFieldCollector.create(sort, Math.max(this.reRankDocs, length), false, true, true, true);
} }
this.searcher = searcher; this.searcher = searcher;
this.reRankQueryRescorer = reRankQueryRescorer; this.reRankQueryRescorer = reRankQueryRescorer;

View File

@ -1518,7 +1518,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
// ... see comments in populateNextCursorMarkFromTopDocs for cache issues (SOLR-5595) // ... see comments in populateNextCursorMarkFromTopDocs for cache issues (SOLR-5595)
final boolean fillFields = (null != cursor); final boolean fillFields = (null != cursor);
final FieldDoc searchAfter = (null != cursor ? cursor.getSearchAfterFieldDoc() : null); final FieldDoc searchAfter = (null != cursor ? cursor.getSearchAfterFieldDoc() : null);
return TopFieldCollector.create(weightedSort, len, searchAfter, fillFields, needScores, needScores); return TopFieldCollector.create(weightedSort, len, searchAfter, fillFields, needScores, needScores, true);
} }
} }

View File

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

View File

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