mirror of https://github.com/apache/lucene.git
LUCENE-8507: TopFieldCollector can now update the minimum competitive score if the primary sort is by relevancy and the total hit count is not required.
This commit is contained in:
parent
a3fc31e5d2
commit
947f82679a
|
@ -183,6 +183,9 @@ Optimizations
|
|||
* LUCENE-8511: MultiFields.getIndexedFields is now optimized; does not call getMergedFieldInfos
|
||||
(David Smiley)
|
||||
|
||||
* LUCENE-8507: TopFieldCollector can now update the minimum competitive score if the primary sort
|
||||
is by relevancy and the total hit count is not required. (Jim Ferenczi)
|
||||
|
||||
======================= Lucene 7.6.0 =======================
|
||||
|
||||
Build:
|
||||
|
|
|
@ -27,7 +27,6 @@ import org.apache.lucene.index.ReaderUtil;
|
|||
import org.apache.lucene.search.FieldValueHitQueue.Entry;
|
||||
import org.apache.lucene.search.TotalHits.Relation;
|
||||
import org.apache.lucene.util.FutureObjects;
|
||||
import org.apache.lucene.util.PriorityQueue;
|
||||
|
||||
/**
|
||||
* A {@link Collector} that sorts by {@link SortField} using
|
||||
|
@ -86,13 +85,11 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
|
|||
|
||||
final Sort sort;
|
||||
final FieldValueHitQueue<Entry> queue;
|
||||
final int totalHitsThreshold;
|
||||
|
||||
public SimpleFieldCollector(Sort sort, FieldValueHitQueue<Entry> queue, int numHits, int totalHitsThreshold) {
|
||||
super(queue, numHits, sort.needsScores());
|
||||
super(queue, numHits, totalHitsThreshold, sort.needsScores());
|
||||
this.sort = sort;
|
||||
this.queue = queue;
|
||||
this.totalHitsThreshold = totalHitsThreshold;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -109,6 +106,12 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
|
|||
|
||||
boolean collectedAllCompetitiveHits = false;
|
||||
|
||||
@Override
|
||||
public void setScorer(Scorable scorer) throws IOException {
|
||||
super.setScorer(scorer);
|
||||
updateMinCompetitiveScore(scorer);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void collect(int doc) throws IOException {
|
||||
++totalHits;
|
||||
|
@ -124,6 +127,10 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
|
|||
} else {
|
||||
collectedAllCompetitiveHits = true;
|
||||
}
|
||||
} else if (totalHitsRelation == Relation.EQUAL_TO) {
|
||||
// we just reached totalHitsThreshold, we can start setting the min
|
||||
// competitive score now
|
||||
updateMinCompetitiveScore(scorer);
|
||||
}
|
||||
return;
|
||||
}
|
||||
|
@ -132,6 +139,7 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
|
|||
comparator.copy(bottom.slot, doc);
|
||||
updateBottom(doc);
|
||||
comparator.setBottom(bottom.slot);
|
||||
updateMinCompetitiveScore(scorer);
|
||||
} else {
|
||||
// Startup transient: queue hasn't gathered numHits yet
|
||||
final int slot = totalHits - 1;
|
||||
|
@ -141,6 +149,7 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
|
|||
add(slot, doc);
|
||||
if (queueFull) {
|
||||
comparator.setBottom(bottom.slot);
|
||||
updateMinCompetitiveScore(scorer);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -163,7 +172,7 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
|
|||
|
||||
public PagingFieldCollector(Sort sort, FieldValueHitQueue<Entry> queue, FieldDoc after, int numHits,
|
||||
int totalHitsThreshold) {
|
||||
super(queue, numHits, sort.needsScores());
|
||||
super(queue, numHits, totalHitsThreshold, sort.needsScores());
|
||||
this.sort = sort;
|
||||
this.queue = queue;
|
||||
this.after = after;
|
||||
|
@ -189,6 +198,12 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
|
|||
|
||||
boolean collectedAllCompetitiveHits = false;
|
||||
|
||||
@Override
|
||||
public void setScorer(Scorable scorer) throws IOException {
|
||||
super.setScorer(scorer);
|
||||
updateMinCompetitiveScore(scorer);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void collect(int doc) throws IOException {
|
||||
//System.out.println(" collect doc=" + doc);
|
||||
|
@ -209,6 +224,8 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
|
|||
} else {
|
||||
collectedAllCompetitiveHits = true;
|
||||
}
|
||||
} else if (totalHitsRelation == Relation.GREATER_THAN_OR_EQUAL_TO) {
|
||||
updateMinCompetitiveScore(scorer);
|
||||
}
|
||||
return;
|
||||
}
|
||||
|
@ -227,6 +244,7 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
|
|||
updateBottom(doc);
|
||||
|
||||
comparator.setBottom(bottom.slot);
|
||||
updateMinCompetitiveScore(scorer);
|
||||
} else {
|
||||
collectedHits++;
|
||||
|
||||
|
@ -240,6 +258,7 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
|
|||
queueFull = collectedHits == numHits;
|
||||
if (queueFull) {
|
||||
comparator.setBottom(bottom.slot);
|
||||
updateMinCompetitiveScore(scorer);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -251,25 +270,54 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
|
|||
private static final ScoreDoc[] EMPTY_SCOREDOCS = new ScoreDoc[0];
|
||||
|
||||
final int numHits;
|
||||
final int totalHitsThreshold;
|
||||
final FieldComparator.RelevanceComparator firstComparator;
|
||||
final boolean canSetMinScore;
|
||||
final int numComparators;
|
||||
FieldValueHitQueue.Entry bottom = null;
|
||||
boolean queueFull;
|
||||
int docBase;
|
||||
final boolean needsScores;
|
||||
final ScoreMode scoreMode;
|
||||
|
||||
// Declaring the constructor private prevents extending this class by anyone
|
||||
// else. Note that the class cannot be final since it's extended by the
|
||||
// internal versions. If someone will define a constructor with any other
|
||||
// visibility, then anyone will be able to extend the class, which is not what
|
||||
// we want.
|
||||
private TopFieldCollector(PriorityQueue<Entry> pq, int numHits, boolean needsScores) {
|
||||
private TopFieldCollector(FieldValueHitQueue<Entry> pq, int numHits, int totalHitsThreshold, boolean needsScores) {
|
||||
super(pq);
|
||||
this.needsScores = needsScores;
|
||||
this.numHits = numHits;
|
||||
this.totalHitsThreshold = totalHitsThreshold;
|
||||
this.numComparators = pq.getComparators().length;
|
||||
FieldComparator<?> fieldComparator = pq.getComparators()[0];
|
||||
int reverseMul = pq.reverseMul[0];
|
||||
if (fieldComparator.getClass().equals(FieldComparator.RelevanceComparator.class)
|
||||
&& reverseMul == 1 // if the natural sort is preserved (sort by descending relevance)
|
||||
&& totalHitsThreshold != Integer.MAX_VALUE) {
|
||||
firstComparator = (FieldComparator.RelevanceComparator) fieldComparator;
|
||||
scoreMode = ScoreMode.TOP_SCORES;
|
||||
canSetMinScore = true;
|
||||
} else {
|
||||
firstComparator = null;
|
||||
scoreMode = needsScores ? ScoreMode.COMPLETE : ScoreMode.COMPLETE_NO_SCORES;
|
||||
canSetMinScore = false;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ScoreMode scoreMode() {
|
||||
return needsScores ? ScoreMode.COMPLETE : ScoreMode.COMPLETE_NO_SCORES;
|
||||
return scoreMode;
|
||||
}
|
||||
|
||||
protected void updateMinCompetitiveScore(Scorable scorer) throws IOException {
|
||||
if (canSetMinScore && totalHits >= totalHitsThreshold && queueFull) {
|
||||
assert bottom != null && firstComparator != null;
|
||||
float minScore = firstComparator.value(bottom.slot);
|
||||
scorer.setMinCompetitiveScore(minScore);
|
||||
totalHitsRelation = TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -44,39 +44,23 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
|
|||
public void setScorer(Scorable scorer) throws IOException {
|
||||
this.scorer = scorer;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private static class SimpleTopScoreDocCollector extends TopScoreDocCollector {
|
||||
|
||||
private final int totalHitsThreshold;
|
||||
|
||||
SimpleTopScoreDocCollector(int numHits, int totalHitsThreshold) {
|
||||
super(numHits);
|
||||
this.totalHitsThreshold = totalHitsThreshold;
|
||||
super(numHits, totalHitsThreshold);
|
||||
}
|
||||
|
||||
@Override
|
||||
public LeafCollector getLeafCollector(LeafReaderContext context)
|
||||
throws IOException {
|
||||
public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
|
||||
final int docBase = context.docBase;
|
||||
return new ScorerLeafCollector() {
|
||||
|
||||
private void updateMinCompetitiveScore() throws IOException {
|
||||
// since we tie-break on doc id and collect in doc id order, we can require
|
||||
// the next float
|
||||
scorer.setMinCompetitiveScore(Math.nextUp(pqTop.score));
|
||||
totalHitsRelation = TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setScorer(Scorable scorer) throws IOException {
|
||||
super.setScorer(scorer);
|
||||
if (totalHits >= totalHitsThreshold
|
||||
&& pqTop != null
|
||||
&& pqTop.score != Float.NEGATIVE_INFINITY) {
|
||||
updateMinCompetitiveScore();
|
||||
}
|
||||
updateMinCompetitiveScore(scorer);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -91,7 +75,7 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
|
|||
if (totalHitsRelation == TotalHits.Relation.EQUAL_TO && totalHits >= totalHitsThreshold) {
|
||||
// we just reached totalHitsThreshold, we can start setting the min
|
||||
// competitive score now
|
||||
updateMinCompetitiveScore();
|
||||
updateMinCompetitiveScore(scorer);
|
||||
}
|
||||
// Since docs are returned in-order (i.e., increasing doc Id), a document
|
||||
// with equal score to pqTop.score cannot compete since HitQueue favors
|
||||
|
@ -101,18 +85,11 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
|
|||
pqTop.doc = doc + docBase;
|
||||
pqTop.score = score;
|
||||
pqTop = pq.updateTop();
|
||||
if (totalHits >= totalHitsThreshold && pqTop.score != Float.NEGATIVE_INFINITY) { // -Infinity is the score of sentinels
|
||||
updateMinCompetitiveScore();
|
||||
}
|
||||
updateMinCompetitiveScore(scorer);
|
||||
}
|
||||
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public ScoreMode scoreMode() {
|
||||
return totalHitsThreshold == Integer.MAX_VALUE ? ScoreMode.COMPLETE : ScoreMode.TOP_SCORES;
|
||||
}
|
||||
}
|
||||
|
||||
private static class PagingTopScoreDocCollector extends TopScoreDocCollector {
|
||||
|
@ -120,8 +97,8 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
|
|||
private final ScoreDoc after;
|
||||
private int collectedHits;
|
||||
|
||||
PagingTopScoreDocCollector(int numHits, ScoreDoc after) {
|
||||
super(numHits);
|
||||
PagingTopScoreDocCollector(int numHits, ScoreDoc after, int totalHitsThreshold) {
|
||||
super(numHits, totalHitsThreshold);
|
||||
this.after = after;
|
||||
this.collectedHits = 0;
|
||||
}
|
||||
|
@ -138,10 +115,12 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
|
|||
: new TopDocs(new TotalHits(totalHits, totalHitsRelation), results);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
|
||||
final int docBase = context.docBase;
|
||||
final int afterDoc = after.doc - context.docBase;
|
||||
|
||||
return new ScorerLeafCollector() {
|
||||
@Override
|
||||
public void collect(int doc) throws IOException {
|
||||
|
@ -154,6 +133,11 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
|
|||
|
||||
if (score > after.score || (score == after.score && doc <= afterDoc)) {
|
||||
// hit was collected on a previous page
|
||||
if (totalHitsRelation == TotalHits.Relation.EQUAL_TO && totalHits >= totalHitsThreshold) {
|
||||
// we just reached totalHitsThreshold, we can start setting the min
|
||||
// competitive score now
|
||||
updateMinCompetitiveScore(scorer);
|
||||
}
|
||||
return;
|
||||
}
|
||||
|
||||
|
@ -167,6 +151,7 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
|
|||
pqTop.doc = doc + docBase;
|
||||
pqTop.score = score;
|
||||
pqTop = pq.updateTop();
|
||||
updateMinCompetitiveScore(scorer);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -220,15 +205,17 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
|
|||
if (after == null) {
|
||||
return new SimpleTopScoreDocCollector(numHits, totalHitsThreshold);
|
||||
} else {
|
||||
return new PagingTopScoreDocCollector(numHits, after);
|
||||
return new PagingTopScoreDocCollector(numHits, after, totalHitsThreshold);
|
||||
}
|
||||
}
|
||||
|
||||
final int totalHitsThreshold;
|
||||
ScoreDoc pqTop;
|
||||
|
||||
// prevents instantiation
|
||||
TopScoreDocCollector(int numHits) {
|
||||
TopScoreDocCollector(int numHits, int totalHitsThreshold) {
|
||||
super(new HitQueue(numHits, true));
|
||||
this.totalHitsThreshold = totalHitsThreshold;
|
||||
// HitQueue implements getSentinelObject to return a ScoreDoc, so we know
|
||||
// that at this point top() is already initialized.
|
||||
pqTop = pq.top();
|
||||
|
@ -245,6 +232,17 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
|
|||
|
||||
@Override
|
||||
public ScoreMode scoreMode() {
|
||||
return ScoreMode.COMPLETE;
|
||||
return totalHitsThreshold == Integer.MAX_VALUE ? ScoreMode.COMPLETE : ScoreMode.TOP_SCORES;
|
||||
}
|
||||
|
||||
protected void updateMinCompetitiveScore(Scorable scorer) throws IOException {
|
||||
if (totalHits >= totalHitsThreshold
|
||||
&& pqTop != null
|
||||
&& pqTop.score != Float.NEGATIVE_INFINITY) { // -Infinity is the score of sentinels
|
||||
// since we tie-break on doc id and collect in doc id order, we can require
|
||||
// the next float
|
||||
scorer.setMinCompetitiveScore(Math.nextUp(pqTop.score));
|
||||
totalHitsRelation = TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -82,7 +82,7 @@ public class TestNeedsScores extends LuceneTestCase {
|
|||
|
||||
/** when sorting by score */
|
||||
public void testSortByScore() throws Exception {
|
||||
Query query = new AssertNeedsScores(new MatchAllDocsQuery(), ScoreMode.COMPLETE);
|
||||
Query query = new AssertNeedsScores(new MatchAllDocsQuery(), ScoreMode.TOP_SCORES);
|
||||
assertEquals(5, searcher.search(query, 5, Sort.RELEVANCE).totalHits.value);
|
||||
}
|
||||
|
||||
|
|
|
@ -39,6 +39,8 @@ import org.apache.lucene.store.Directory;
|
|||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
|
||||
import static org.apache.lucene.search.SortField.FIELD_SCORE;
|
||||
|
||||
public class TestTopFieldCollector extends LuceneTestCase {
|
||||
private IndexSearcher is;
|
||||
private IndexReader ir;
|
||||
|
@ -199,6 +201,138 @@ public class TestTopFieldCollector extends LuceneTestCase {
|
|||
dir.close();
|
||||
}
|
||||
|
||||
private static class ScoreAndDoc extends Scorable {
|
||||
int doc = -1;
|
||||
float score;
|
||||
Float minCompetitiveScore = null;
|
||||
|
||||
@Override
|
||||
public void setMinCompetitiveScore(float minCompetitiveScore) {
|
||||
this.minCompetitiveScore = minCompetitiveScore;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docID() {
|
||||
return doc;
|
||||
}
|
||||
|
||||
@Override
|
||||
public float score() throws IOException {
|
||||
return score;
|
||||
}
|
||||
}
|
||||
|
||||
public void testSetMinCompetitiveScore() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter w = new IndexWriter(dir, newIndexWriterConfig().setMergePolicy(NoMergePolicy.INSTANCE));
|
||||
Document doc = new Document();
|
||||
w.addDocuments(Arrays.asList(doc, doc, doc, doc));
|
||||
w.flush();
|
||||
w.addDocuments(Arrays.asList(doc, doc));
|
||||
w.flush();
|
||||
IndexReader reader = DirectoryReader.open(w);
|
||||
assertEquals(2, reader.leaves().size());
|
||||
w.close();
|
||||
|
||||
Sort sort = new Sort(FIELD_SCORE, new SortField("foo", SortField.Type.LONG));
|
||||
TopFieldCollector collector = TopFieldCollector.create(sort, 2, null, 1);
|
||||
ScoreAndDoc scorer = new ScoreAndDoc();
|
||||
|
||||
LeafCollector leafCollector = collector.getLeafCollector(reader.leaves().get(0));
|
||||
leafCollector.setScorer(scorer);
|
||||
assertNull(scorer.minCompetitiveScore);
|
||||
|
||||
scorer.doc = 0;
|
||||
scorer.score = 1;
|
||||
leafCollector.collect(0);
|
||||
assertNull(scorer.minCompetitiveScore);
|
||||
|
||||
scorer.doc = 1;
|
||||
scorer.score = 2;
|
||||
leafCollector.collect(1);
|
||||
assertEquals(1f, scorer.minCompetitiveScore, 0f);
|
||||
|
||||
scorer.doc = 2;
|
||||
scorer.score = 0.5f;
|
||||
// Make sure we do not call setMinCompetitiveScore for non-competitive hits
|
||||
scorer.minCompetitiveScore = Float.NaN;
|
||||
leafCollector.collect(2);
|
||||
assertTrue(Float.isNaN(scorer.minCompetitiveScore));
|
||||
|
||||
scorer.doc = 3;
|
||||
scorer.score = 4;
|
||||
leafCollector.collect(3);
|
||||
assertEquals(2f, scorer.minCompetitiveScore, 0f);
|
||||
|
||||
// Make sure the min score is set on scorers on new segments
|
||||
scorer = new ScoreAndDoc();
|
||||
leafCollector = collector.getLeafCollector(reader.leaves().get(1));
|
||||
leafCollector.setScorer(scorer);
|
||||
assertEquals(2f, scorer.minCompetitiveScore, 0f);
|
||||
|
||||
scorer.doc = 0;
|
||||
scorer.score = 1;
|
||||
leafCollector.collect(0);
|
||||
assertEquals(2f, scorer.minCompetitiveScore, 0f);
|
||||
|
||||
scorer.doc = 1;
|
||||
scorer.score = 3;
|
||||
leafCollector.collect(1);
|
||||
assertEquals(3f, scorer.minCompetitiveScore, 0f);
|
||||
|
||||
reader.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public void testTotalHitsWithScore() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter w = new IndexWriter(dir, newIndexWriterConfig().setMergePolicy(NoMergePolicy.INSTANCE));
|
||||
Document doc = new Document();
|
||||
w.addDocuments(Arrays.asList(doc, doc, doc, doc));
|
||||
w.flush();
|
||||
w.addDocuments(Arrays.asList(doc, doc, doc, doc, doc, doc));
|
||||
w.flush();
|
||||
IndexReader reader = DirectoryReader.open(w);
|
||||
assertEquals(2, reader.leaves().size());
|
||||
w.close();
|
||||
|
||||
for (int totalHitsThreshold = 1; totalHitsThreshold < 20; ++ totalHitsThreshold) {
|
||||
Sort sort = new Sort(FIELD_SCORE, new SortField("foo", SortField.Type.LONG));
|
||||
TopFieldCollector collector = TopFieldCollector.create(sort, 2, null, totalHitsThreshold);
|
||||
ScoreAndDoc scorer = new ScoreAndDoc();
|
||||
|
||||
LeafCollector leafCollector = collector.getLeafCollector(reader.leaves().get(0));
|
||||
leafCollector.setScorer(scorer);
|
||||
|
||||
scorer.doc = 0;
|
||||
scorer.score = 3;
|
||||
leafCollector.collect(0);
|
||||
|
||||
scorer.doc = 1;
|
||||
scorer.score = 3;
|
||||
leafCollector.collect(1);
|
||||
|
||||
leafCollector = collector.getLeafCollector(reader.leaves().get(1));
|
||||
leafCollector.setScorer(scorer);
|
||||
|
||||
scorer.doc = 1;
|
||||
scorer.score = 3;
|
||||
leafCollector.collect(1);
|
||||
|
||||
scorer.doc = 5;
|
||||
scorer.score = 4;
|
||||
leafCollector.collect(1);
|
||||
|
||||
TopDocs topDocs = collector.topDocs();
|
||||
assertEquals(4, topDocs.totalHits.value);
|
||||
assertEquals(totalHitsThreshold <= 4, scorer.minCompetitiveScore != null);
|
||||
assertEquals(totalHitsThreshold <= 4 ? TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO : TotalHits.Relation.EQUAL_TO, topDocs.totalHits.relation);
|
||||
}
|
||||
|
||||
reader.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public void testSortNoResults() throws Exception {
|
||||
|
||||
// Two Sort criteria to instantiate the multi/single comparators.
|
||||
|
@ -235,7 +369,7 @@ public class TestTopFieldCollector extends LuceneTestCase {
|
|||
.add(baz, Occur.SHOULD)
|
||||
.build();
|
||||
final IndexSearcher searcher = new IndexSearcher(reader);
|
||||
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(FIELD_SCORE), new Sort(new SortField("f", SortField.Type.SCORE))}) {
|
||||
final TopFieldCollector topCollector = TopFieldCollector.create(sort, TestUtil.nextInt(random(), 1, 2), Integer.MAX_VALUE);
|
||||
final Collector assertingCollector = new Collector() {
|
||||
@Override
|
||||
|
|
Loading…
Reference in New Issue