You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ab...@apache.org on 2017/12/04 17:49:16 UTC
[32/50] lucene-solr:jira/solr-11458-2: LUCENE-8059: Fold early
termination support into TopFieldCollector.
LUCENE-8059: Fold early termination support into TopFieldCollector.
Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/81271141
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/81271141
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/81271141
Branch: refs/heads/jira/solr-11458-2
Commit: 812711416d903be47a2bf14c64aabf67e0753a5d
Parents: d27ddcb
Author: Adrien Grand <jp...@gmail.com>
Authored: Wed Nov 29 16:07:33 2017 +0100
Committer: Adrien Grand <jp...@gmail.com>
Committed: Wed Nov 29 18:09:38 2017 +0100
----------------------------------------------------------------------
lucene/CHANGES.txt | 4 +
.../lucene/benchmark/byTask/tasks/ReadTask.java | 9 +-
.../EarlyTerminatingSortingCollector.java | 129 ----------
.../org/apache/lucene/search/IndexSearcher.java | 3 +-
.../org/apache/lucene/search/SortRescorer.java | 2 +-
.../apache/lucene/search/TopFieldCollector.java | 86 ++++++-
.../apache/lucene/index/TestIndexSorting.java | 10 +-
.../org/apache/lucene/search/TestBoolean2.java | 8 +-
.../TestEarlyTerminatingSortingCollector.java | 246 -------------------
.../lucene/search/TestElevationComparator.java | 2 +-
.../apache/lucene/search/TestTopDocsMerge.java | 4 +-
.../lucene/search/TestTopFieldCollector.java | 14 +-
.../TestTopFieldCollectorEarlyTermination.java | 213 ++++++++++++++++
.../org/apache/lucene/facet/DrillSideways.java | 4 +-
.../apache/lucene/facet/FacetsCollector.java | 3 +-
.../search/grouping/BlockGroupingCollector.java | 2 +-
.../search/grouping/TopGroupsCollector.java | 2 +-
.../analyzing/AnalyzingInfixSuggester.java | 10 +-
.../solr/handler/component/ExpandComponent.java | 4 +-
.../EarlyTerminatingSortingCollector.java | 132 ++++++++++
.../java/org/apache/solr/search/Grouping.java | 2 +-
.../org/apache/solr/search/ReRankCollector.java | 2 +-
.../apache/solr/search/SolrIndexSearcher.java | 2 +-
.../distributed/command/QueryCommand.java | 2 +-
.../test/org/apache/solr/search/TestSort.java | 2 +-
25 files changed, 469 insertions(+), 428 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81271141/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 04cfde1..6b4aa9c 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -73,6 +73,10 @@ API Changes
* LUCENE-8049: IndexWriter.getMergingSegments()'s return type was changed from
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
* LUCENE-8061: Add convenience factory methods to create BBoxes and XYZSolids
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81271141/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ReadTask.java
----------------------------------------------------------------------
diff --git a/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ReadTask.java b/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ReadTask.java
index 4ffad3d..a3b90f5 100644
--- a/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ReadTask.java
+++ b/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ReadTask.java
@@ -113,7 +113,8 @@ public abstract class ReadTask extends PerfTask {
// pulling the Weight ourselves:
TopFieldCollector collector = TopFieldCollector.create(sort, numHits,
true, withScore(),
- withMaxScore());
+ withMaxScore(),
+ withTotalHits());
searcher.search(q, collector);
hits = collector.topDocs();
} else {
@@ -220,6 +221,12 @@ public abstract class ReadTask extends PerfTask {
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
* of hits that are traversed when {@link #withTraverse()} is true. Must be greater than 0.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81271141/lucene/core/src/java/org/apache/lucene/search/EarlyTerminatingSortingCollector.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/EarlyTerminatingSortingCollector.java b/lucene/core/src/java/org/apache/lucene/search/EarlyTerminatingSortingCollector.java
deleted file mode 100644
index 623df65..0000000
--- a/lucene/core/src/java/org/apache/lucene/search/EarlyTerminatingSortingCollector.java
+++ /dev/null
@@ -1,129 +0,0 @@
-/*
- * 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.
- */
-package org.apache.lucene.search;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.search.CollectionTerminatedException;
-import org.apache.lucene.search.Collector;
-import org.apache.lucene.search.FilterCollector;
-import org.apache.lucene.search.FilterLeafCollector;
-import org.apache.lucene.search.LeafCollector;
-import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.TopDocsCollector;
-import org.apache.lucene.search.TotalHitCountCollector;
-
-/**
- * A {@link Collector} that early terminates collection of documents on a
- * per-segment basis, if the segment was sorted according to the given
- * {@link Sort}.
- *
- * <p>
- * <b>NOTE:</b> the {@code Collector} detects segments sorted according to a
- * an {@link IndexWriterConfig#setIndexSort}. Also, it collects up to a specified
- * {@code numDocsToCollect} from each segment, and therefore is mostly suitable
- * for use in conjunction with collectors such as {@link TopDocsCollector}, and
- * not e.g. {@link TotalHitCountCollector}.
- * <p>
- * <b>NOTE</b>: If you wrap a {@code TopDocsCollector} that sorts in the same
- * order as the index order, the returned {@link TopDocsCollector#topDocs() TopDocs}
- * will be correct. However the total of {@link TopDocsCollector#getTotalHits()
- * hit count} will be vastly underestimated since not all matching documents will have
- * been collected.
- *
- * @lucene.experimental
- */
-
-public class EarlyTerminatingSortingCollector extends FilterCollector {
-
- /** Returns whether collection can be early-terminated if it sorts with the
- * provided {@link Sort} and if segments are merged with the provided
- * {@link Sort}. */
- public static boolean canEarlyTerminate(Sort searchSort, Sort mergePolicySort) {
- final SortField[] fields1 = searchSort.getSort();
- final SortField[] fields2 = mergePolicySort.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));
- }
-
- /** Sort used to sort the search results */
- protected final Sort sort;
- /** Number of documents to collect in each segment */
- protected final int numDocsToCollect;
- private final AtomicBoolean terminatedEarly = new AtomicBoolean(false);
-
- /**
- * Create a new {@link EarlyTerminatingSortingCollector} instance.
- *
- * @param in
- * the collector to wrap
- * @param sort
- * the sort you are sorting the search results on
- * @param numDocsToCollect
- * the number of documents to collect on each segment. When wrapping
- * a {@link TopDocsCollector}, this number should be the number of
- * hits.
- * @throws IllegalArgumentException if the sort order doesn't allow for early
- * termination with the given merge policy.
- */
- public EarlyTerminatingSortingCollector(Collector in, Sort sort, int numDocsToCollect) {
- super(in);
- if (numDocsToCollect <= 0) {
- throw new IllegalArgumentException("numDocsToCollect must always be > 0, got " + numDocsToCollect);
- }
- this.sort = sort;
- this.numDocsToCollect = numDocsToCollect;
- }
-
- @Override
- public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
- Sort segmentSort = context.reader().getMetaData().getSort();
- if (segmentSort != null && canEarlyTerminate(sort, segmentSort) == false) {
- throw new IllegalStateException("Cannot early terminate with sort order " + sort + " if segments are sorted with " + segmentSort);
- }
-
- if (segmentSort != null) {
- // segment is sorted, can early-terminate
- return new FilterLeafCollector(super.getLeafCollector(context)) {
- private int numCollected;
-
- @Override
- public void collect(int doc) throws IOException {
- super.collect(doc);
- if (++numCollected >= numDocsToCollect) {
- terminatedEarly.set(true);
- throw new CollectionTerminatedException();
- }
- }
-
- };
- } else {
- return super.getLeafCollector(context);
- }
- }
-
- public boolean terminatedEarly() {
- return terminatedEarly.get();
- }
-}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81271141/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java b/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
index 5cb88b6..accb99e 100644
--- a/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
+++ b/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
@@ -535,7 +535,8 @@ public class IndexSearcher {
@Override
public TopFieldCollector newCollector() throws IOException {
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
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81271141/lucene/core/src/java/org/apache/lucene/search/SortRescorer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/SortRescorer.java b/lucene/core/src/java/org/apache/lucene/search/SortRescorer.java
index f051d38..d3912dd 100644
--- a/lucene/core/src/java/org/apache/lucene/search/SortRescorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/SortRescorer.java
@@ -54,7 +54,7 @@ public class SortRescorer extends Rescorer {
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:
int hitUpto = 0;
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81271141/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java b/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
index 97589c4..94f156e 100644
--- a/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
+++ b/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
@@ -18,6 +18,7 @@ package org.apache.lucene.search;
import java.io.IOException;
+import java.util.Arrays;
import org.apache.lucene.index.LeafReaderContext;
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
* {@link FieldComparator}s.
* <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.
*
* @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
* document scores and maxScore.
*/
private static class SimpleFieldCollector extends TopFieldCollector {
+ final Sort sort;
final FieldValueHitQueue<Entry> queue;
final boolean trackDocScores;
final boolean trackMaxScore;
final boolean mayNeedScoresTwice;
+ final boolean trackTotalHits;
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);
+ this.sort = sort;
this.queue = queue;
if (trackMaxScore) {
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
// to cache scores would help
this.mayNeedScoresTwice = sort.needsScores() && (trackDocScores || trackMaxScore);
+ this.trackTotalHits = trackTotalHits;
}
@Override
@@ -99,6 +121,10 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
final LeafFieldComparator[] comparators = queue.getComparators(context);
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) {
@@ -118,7 +144,16 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
// 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
// therefore not competitive.
- return;
+ 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;
+ }
}
if (trackDocScores && !trackMaxScore) {
@@ -156,21 +191,25 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
*/
private final static class PagingFieldCollector extends TopFieldCollector {
+ final Sort sort;
int collectedHits;
final FieldValueHitQueue<Entry> queue;
final boolean trackDocScores;
final boolean trackMaxScore;
final FieldDoc after;
final boolean mayNeedScoresTwice;
+ final boolean trackTotalHits;
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());
+ this.sort = sort;
this.queue = queue;
this.trackDocScores = trackDocScores;
this.trackMaxScore = trackMaxScore;
this.after = after;
this.mayNeedScoresTwice = sort.needsScores() && (trackDocScores || trackMaxScore);
+ this.trackTotalHits = trackTotalHits;
// Must set maxScore to NEG_INF, or otherwise Math.max always returns NaN.
maxScore = Float.NEGATIVE_INFINITY;
@@ -188,6 +227,10 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
docBase = context.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) {
@Override
@@ -210,7 +253,16 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
final int cmp = reverseMul * comparator.compareBottom(doc);
if (cmp <= 0) {
// not competitive since documents are visited in doc id order
- return;
+ 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;
+ }
}
}
@@ -270,6 +322,7 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
FieldValueHitQueue.Entry bottom = null;
boolean queueFull;
int docBase;
+ boolean earlyTerminated = false;
final boolean needsScores;
// 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
* each result. Also, setting this true automatically sets
* <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
* the sort criteria.
*/
public static TopFieldCollector create(Sort sort, int numHits,
- boolean fillFields, boolean trackDocScores, boolean trackMaxScore) {
- return create(sort, numHits, null, fillFields, trackDocScores, trackMaxScore);
+ boolean fillFields, boolean trackDocScores, boolean trackMaxScore, boolean trackTotalHits) {
+ 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
* each result. Also, setting this true automatically sets
* <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
* the sort criteria.
*/
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) {
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);
if (after == null) {
- return new SimpleFieldCollector(sort, queue, numHits, fillFields, trackDocScores, trackMaxScore);
+ return new SimpleFieldCollector(sort, queue, numHits, fillFields, trackDocScores, trackMaxScore, trackTotalHits);
} else {
if (after.fields == null) {
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);
}
- 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() {
return (TopFieldDocs) super.topDocs();
}
+
+ /** Return whether collection terminated early. */
+ public boolean isEarlyTerminated() {
+ return earlyTerminated;
+ }
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81271141/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java
index bd483d3..0a8799d 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java
@@ -61,7 +61,6 @@ import org.apache.lucene.document.StoredField;
import org.apache.lucene.document.StringField;
import org.apache.lucene.document.TextField;
import org.apache.lucene.search.CollectionStatistics;
-import org.apache.lucene.search.EarlyTerminatingSortingCollector;
import org.apache.lucene.search.FieldDoc;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.MatchAllDocsQuery;
@@ -2335,13 +2334,12 @@ public class TestIndexSorting extends LuceneTestCase {
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);
TopDocs hits1 = c1.topDocs();
- TopFieldCollector c2 = TopFieldCollector.create(sort, numHits, true, true, true);
- EarlyTerminatingSortingCollector c3 = new EarlyTerminatingSortingCollector(c2, sort, numHits);
- s2.search(new MatchAllDocsQuery(), c3);
+ TopFieldCollector c2 = TopFieldCollector.create(sort, numHits, true, true, true, false);
+ s2.search(new MatchAllDocsQuery(), c2);
TopDocs hits2 = c2.topDocs();
@@ -2362,7 +2360,7 @@ public class TestIndexSorting extends LuceneTestCase {
ScoreDoc hit1 = hits1.scoreDocs[i];
ScoreDoc hit2 = hits2.scoreDocs[i];
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);
}
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81271141/lucene/core/src/test/org/apache/lucene/search/TestBoolean2.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestBoolean2.java b/lucene/core/src/test/org/apache/lucene/search/TestBoolean2.java
index 0974563..9556281 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestBoolean2.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestBoolean2.java
@@ -386,10 +386,10 @@ public class TestBoolean2 extends LuceneTestCase {
}
// 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);
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);
ScoreDoc[] hits2 = collector.topDocs().scoreDocs;
tot+=hits2.length;
@@ -402,10 +402,10 @@ public class TestBoolean2 extends LuceneTestCase {
assertEquals(mulFactor*collector.totalHits + NUM_EXTRA_DOCS/2, hits4.totalHits);
// 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);
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);
hits2 = collector.topDocs().scoreDocs;
CheckHits.checkEqual(q1, hits1, hits2);
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81271141/lucene/core/src/test/org/apache/lucene/search/TestEarlyTerminatingSortingCollector.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestEarlyTerminatingSortingCollector.java b/lucene/core/src/test/org/apache/lucene/search/TestEarlyTerminatingSortingCollector.java
deleted file mode 100644
index 217834c..0000000
--- a/lucene/core/src/test/org/apache/lucene/search/TestEarlyTerminatingSortingCollector.java
+++ /dev/null
@@ -1,246 +0,0 @@
-/*
- * 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.
- */
-package org.apache.lucene.search;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Random;
-import java.util.Set;
-
-import org.apache.lucene.analysis.MockAnalyzer;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field.Store;
-import org.apache.lucene.document.NumericDocValuesField;
-import org.apache.lucene.document.StringField;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.index.MockRandomMergePolicy;
-import org.apache.lucene.index.QueryTimeout;
-import org.apache.lucene.index.RandomIndexWriter;
-import org.apache.lucene.index.SerialMergeScheduler;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.TestUtil;
-
-import com.carrotsearch.randomizedtesting.generators.RandomPicks;
-
-public class TestEarlyTerminatingSortingCollector extends LuceneTestCase {
-
- private int numDocs;
- private List<String> terms;
- private Directory dir;
- private final Sort sort = new Sort(new SortField("ndv1", SortField.Type.LONG));
- private RandomIndexWriter iw;
- private IndexReader reader;
- private static final int FORCE_MERGE_MAX_SEGMENT_COUNT = 5;
-
- private Document randomDocument() {
- final Document doc = new Document();
- doc.add(new NumericDocValuesField("ndv1", random().nextInt(10)));
- doc.add(new NumericDocValuesField("ndv2", random().nextInt(10)));
- doc.add(new StringField("s", RandomPicks.randomFrom(random(), terms), Store.YES));
- return doc;
- }
-
- private void createRandomIndex(boolean singleSortedSegment) throws IOException {
- dir = newDirectory();
- numDocs = atLeast(150);
- final int numTerms = TestUtil.nextInt(random(), 1, numDocs / 5);
- Set<String> randomTerms = new HashSet<>();
- while (randomTerms.size() < numTerms) {
- randomTerms.add(TestUtil.randomSimpleString(random()));
- }
- terms = new ArrayList<>(randomTerms);
- final long seed = random().nextLong();
- final IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(new Random(seed)));
- if (iwc.getMergePolicy() instanceof MockRandomMergePolicy) {
- // MockRandomMP randomly wraps the leaf readers which makes merging angry
- iwc.setMergePolicy(newTieredMergePolicy());
- }
- iwc.setMergeScheduler(new SerialMergeScheduler()); // for reproducible tests
- iwc.setIndexSort(sort);
- iw = new RandomIndexWriter(new Random(seed), dir, iwc);
- iw.setDoRandomForceMerge(false); // don't do this, it may happen anyway with MockRandomMP
- for (int i = 0; i < numDocs; ++i) {
- final Document doc = randomDocument();
- iw.addDocument(doc);
- if (i == numDocs / 2 || (i != numDocs - 1 && random().nextInt(8) == 0)) {
- iw.commit();
- }
- if (random().nextInt(15) == 0) {
- final String term = RandomPicks.randomFrom(random(), terms);
- iw.deleteDocuments(new Term("s", term));
- }
- }
- if (singleSortedSegment) {
- // because of deletions, there might still be a single flush segment in
- // the index, although want want a sorted segment so it needs to be merged
- iw.getReader().close(); // refresh
- iw.addDocument(new Document());
- iw.commit();
- iw.addDocument(new Document());
- iw.forceMerge(1);
- }
- else if (random().nextBoolean()) {
- iw.forceMerge(FORCE_MERGE_MAX_SEGMENT_COUNT);
- }
- reader = iw.getReader();
- }
-
- private void closeIndex() throws IOException {
- reader.close();
- iw.close();
- dir.close();
- }
-
- public void testEarlyTermination() throws IOException {
- final int iters = atLeast(8);
- for (int i = 0; i < iters; ++i) {
- createRandomIndex(false);
- for (int j = 0; j < iters; ++j) {
- final IndexSearcher searcher = newSearcher(reader);
- final int numHits = TestUtil.nextInt(random(), 1, numDocs);
- final Sort sort = new Sort(new SortField("ndv1", SortField.Type.LONG, false));
- final boolean fillFields = random().nextBoolean();
- final boolean trackDocScores = random().nextBoolean();
- final boolean trackMaxScore = random().nextBoolean();
- 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()) {
- query = new TermQuery(new Term("s", RandomPicks.randomFrom(random(), terms)));
- } else {
- query = new MatchAllDocsQuery();
- }
- searcher.search(query, collector1);
- searcher.search(query, new EarlyTerminatingSortingCollector(collector2, sort, numHits));
- assertTrue(collector1.getTotalHits() >= collector2.getTotalHits());
- assertTopDocsEquals(collector1.topDocs().scoreDocs, collector2.topDocs().scoreDocs);
- }
- closeIndex();
- }
- }
-
- public void testCanEarlyTerminate() {
- assertTrue(EarlyTerminatingSortingCollector.canEarlyTerminate(
- new Sort(new SortField("a", SortField.Type.LONG)),
- new Sort(new SortField("a", SortField.Type.LONG))));
-
- assertTrue(EarlyTerminatingSortingCollector.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))));
-
- assertTrue(EarlyTerminatingSortingCollector.canEarlyTerminate(
- new Sort(new SortField("a", SortField.Type.LONG)),
- new Sort(new SortField("a", SortField.Type.LONG), new SortField("b", SortField.Type.STRING))));
-
- assertFalse(EarlyTerminatingSortingCollector.canEarlyTerminate(
- new Sort(new SortField("a", SortField.Type.LONG, true)),
- new Sort(new SortField("a", SortField.Type.LONG, false))));
-
- assertFalse(EarlyTerminatingSortingCollector.canEarlyTerminate(
- new Sort(new SortField("a", SortField.Type.LONG), new SortField("b", SortField.Type.STRING)),
- new Sort(new SortField("a", SortField.Type.LONG))));
-
- assertFalse(EarlyTerminatingSortingCollector.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("c", SortField.Type.STRING))));
-
- assertFalse(EarlyTerminatingSortingCollector.canEarlyTerminate(
- 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))));
- }
-
- 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) {
- assertEquals(scoreDocs1.length, scoreDocs2.length);
- for (int i = 0; i < scoreDocs1.length; ++i) {
- final ScoreDoc scoreDoc1 = scoreDocs1[i];
- final ScoreDoc scoreDoc2 = scoreDocs2[i];
- assertEquals(scoreDoc1.doc, scoreDoc2.doc);
- assertEquals(scoreDoc1.score, scoreDoc2.score, 0.001f);
- }
- }
-
- 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();
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81271141/lucene/core/src/test/org/apache/lucene/search/TestElevationComparator.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestElevationComparator.java b/lucene/core/src/test/org/apache/lucene/search/TestElevationComparator.java
index bc849e9..3178c35 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestElevationComparator.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestElevationComparator.java
@@ -86,7 +86,7 @@ public class TestElevationComparator extends LuceneTestCase {
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);
TopDocs topDocs = topCollector.topDocs(0, 10);
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81271141/lucene/core/src/test/org/apache/lucene/search/TestTopDocsMerge.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestTopDocsMerge.java b/lucene/core/src/test/org/apache/lucene/search/TestTopDocsMerge.java
index 0372c2a..9759863 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestTopDocsMerge.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestTopDocsMerge.java
@@ -281,7 +281,7 @@ public class TestTopDocsMerge extends LuceneTestCase {
topHits = searcher.search(query, numHits);
}
} 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);
if (useFrom) {
from = TestUtil.nextInt(random(), 0, numHits - 1);
@@ -330,7 +330,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);
+ final TopFieldCollector c = TopFieldCollector.create(sort, numHits, true, true, true, true);
subSearcher.search(w, c);
subHits = c.topDocs(0, numHits);
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81271141/lucene/core/src/test/org/apache/lucene/search/TestTopFieldCollector.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestTopFieldCollector.java b/lucene/core/src/test/org/apache/lucene/search/TestTopFieldCollector.java
index d91f4ac..d9be584 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestTopFieldCollector.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestTopFieldCollector.java
@@ -71,7 +71,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);
+ false, false, true);
is.search(q, tdc);
@@ -90,7 +90,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);
+ false, true);
is.search(q, tdc);
@@ -110,7 +110,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);
+ false, true);
is.search(q, tdc);
@@ -131,7 +131,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);
+ false, true);
is.search(q, tdc);
@@ -151,7 +151,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);
@@ -169,7 +169,7 @@ public class TestTopFieldCollector extends LuceneTestCase {
// 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);
+ TopDocsCollector<Entry> tdc = TopFieldCollector.create(sort[i], 10, true, true, true, true);
TopDocs td = tdc.topDocs();
assertEquals(0, td.totalHits);
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 (boolean doDocScores : 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() {
@Override
public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81271141/lucene/core/src/test/org/apache/lucene/search/TestTopFieldCollectorEarlyTermination.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestTopFieldCollectorEarlyTermination.java b/lucene/core/src/test/org/apache/lucene/search/TestTopFieldCollectorEarlyTermination.java
new file mode 100644
index 0000000..91f1720
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/search/TestTopFieldCollectorEarlyTermination.java
@@ -0,0 +1,213 @@
+/*
+ * 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.
+ */
+package org.apache.lucene.search;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field.Store;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.MockRandomMergePolicy;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.SerialMergeScheduler;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+
+import com.carrotsearch.randomizedtesting.generators.RandomPicks;
+
+public class TestTopFieldCollectorEarlyTermination extends LuceneTestCase {
+
+ private int numDocs;
+ private List<String> terms;
+ private Directory dir;
+ private final Sort sort = new Sort(new SortField("ndv1", SortField.Type.LONG));
+ private RandomIndexWriter iw;
+ private IndexReader reader;
+ private static final int FORCE_MERGE_MAX_SEGMENT_COUNT = 5;
+
+ private Document randomDocument() {
+ final Document doc = new Document();
+ doc.add(new NumericDocValuesField("ndv1", random().nextInt(10)));
+ doc.add(new NumericDocValuesField("ndv2", random().nextInt(10)));
+ doc.add(new StringField("s", RandomPicks.randomFrom(random(), terms), Store.YES));
+ return doc;
+ }
+
+ private void createRandomIndex(boolean singleSortedSegment) throws IOException {
+ dir = newDirectory();
+ numDocs = atLeast(150);
+ final int numTerms = TestUtil.nextInt(random(), 1, numDocs / 5);
+ Set<String> randomTerms = new HashSet<>();
+ while (randomTerms.size() < numTerms) {
+ randomTerms.add(TestUtil.randomSimpleString(random()));
+ }
+ terms = new ArrayList<>(randomTerms);
+ final long seed = random().nextLong();
+ final IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(new Random(seed)));
+ if (iwc.getMergePolicy() instanceof MockRandomMergePolicy) {
+ // MockRandomMP randomly wraps the leaf readers which makes merging angry
+ iwc.setMergePolicy(newTieredMergePolicy());
+ }
+ iwc.setMergeScheduler(new SerialMergeScheduler()); // for reproducible tests
+ iwc.setIndexSort(sort);
+ iw = new RandomIndexWriter(new Random(seed), dir, iwc);
+ iw.setDoRandomForceMerge(false); // don't do this, it may happen anyway with MockRandomMP
+ for (int i = 0; i < numDocs; ++i) {
+ final Document doc = randomDocument();
+ iw.addDocument(doc);
+ if (i == numDocs / 2 || (i != numDocs - 1 && random().nextInt(8) == 0)) {
+ iw.commit();
+ }
+ if (random().nextInt(15) == 0) {
+ final String term = RandomPicks.randomFrom(random(), terms);
+ iw.deleteDocuments(new Term("s", term));
+ }
+ }
+ if (singleSortedSegment) {
+ // because of deletions, there might still be a single flush segment in
+ // the index, although want want a sorted segment so it needs to be merged
+ iw.getReader().close(); // refresh
+ iw.addDocument(new Document());
+ iw.commit();
+ iw.addDocument(new Document());
+ iw.forceMerge(1);
+ }
+ else if (random().nextBoolean()) {
+ iw.forceMerge(FORCE_MERGE_MAX_SEGMENT_COUNT);
+ }
+ reader = iw.getReader();
+ }
+
+ private void closeIndex() throws IOException {
+ reader.close();
+ iw.close();
+ dir.close();
+ }
+
+ 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);
+ for (int i = 0; i < iters; ++i) {
+ createRandomIndex(false);
+ int maxSegmentSize = 0;
+ for (LeafReaderContext ctx : reader.leaves()) {
+ maxSegmentSize = Math.max(ctx.reader().numDocs(), maxSegmentSize);
+ }
+ for (int j = 0; j < iters; ++j) {
+ final IndexSearcher searcher = newSearcher(reader);
+ final int numHits = TestUtil.nextInt(random(), 1, numDocs);
+ 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 trackDocScores = random().nextBoolean();
+ final boolean trackMaxScore = random().nextBoolean();
+ final TopFieldCollector collector1 = TopFieldCollector.create(sort, numHits, after, fillFields, trackDocScores, trackMaxScore, true);
+ final TopFieldCollector collector2 = TopFieldCollector.create(sort, numHits, after, fillFields, trackDocScores, trackMaxScore, false);
+
+ final Query query;
+ if (random().nextBoolean()) {
+ query = new TermQuery(new Term("s", RandomPicks.randomFrom(random(), terms)));
+ } else {
+ query = new MatchAllDocsQuery();
+ }
+ searcher.search(query, collector1);
+ searcher.search(query, collector2);
+ TopDocs td1 = collector1.topDocs();
+ 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();
+ }
+ }
+
+ public void testCanEarlyTerminate() {
+ assertTrue(TopFieldCollector.canEarlyTerminate(
+ new Sort(new SortField("a", SortField.Type.LONG)),
+ new Sort(new SortField("a", SortField.Type.LONG))));
+
+ 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))));
+
+ assertTrue(TopFieldCollector.canEarlyTerminate(
+ new Sort(new SortField("a", SortField.Type.LONG)),
+ new Sort(new SortField("a", SortField.Type.LONG), new SortField("b", SortField.Type.STRING))));
+
+ assertFalse(TopFieldCollector.canEarlyTerminate(
+ new Sort(new SortField("a", SortField.Type.LONG, true)),
+ new Sort(new SortField("a", SortField.Type.LONG, false))));
+
+ 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))));
+
+ 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("c", SortField.Type.STRING))));
+
+ assertFalse(TopFieldCollector.canEarlyTerminate(
+ 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))));
+ }
+
+ private static void assertTopDocsEquals(ScoreDoc[] scoreDocs1, ScoreDoc[] scoreDocs2) {
+ assertEquals(scoreDocs1.length, scoreDocs2.length);
+ for (int i = 0; i < scoreDocs1.length; ++i) {
+ final ScoreDoc scoreDoc1 = scoreDocs1[i];
+ final ScoreDoc scoreDoc2 = scoreDocs2[i];
+ assertEquals(scoreDoc1.doc, scoreDoc2.doc);
+ assertEquals(scoreDoc1.score, scoreDoc2.score, 0f);
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81271141/lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java b/lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java
index b2391f5..fec10b0 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java
@@ -240,7 +240,7 @@ public class DrillSideways {
@Override
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
@@ -259,7 +259,7 @@ public class DrillSideways {
} else {
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);
return new DrillSidewaysResult(r.facets, hitCollector.topDocs());
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81271141/lucene/facet/src/java/org/apache/lucene/facet/FacetsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/FacetsCollector.java b/lucene/facet/src/java/org/apache/lucene/facet/FacetsCollector.java
index 9333353..3cf56e2 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/FacetsCollector.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/FacetsCollector.java
@@ -232,7 +232,8 @@ public class FacetsCollector extends SimpleCollector implements Collector {
(FieldDoc) after,
fillFields,
doDocScores,
- doMaxScore);
+ doMaxScore,
+ true); // TODO: can we disable exact hit counts
} else {
hitsCollector = TopScoreDocCollector.create(n, after);
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81271141/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java
index a50fda1..baab845 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java
@@ -305,7 +305,7 @@ public class BlockGroupingCollector extends SimpleCollector {
collector = TopScoreDocCollector.create(maxDocsPerGroup);
} else {
// 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);
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81271141/lucene/grouping/src/java/org/apache/lucene/search/grouping/TopGroupsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/TopGroupsCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/TopGroupsCollector.java
index b6c71d3..7e41a6c 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/TopGroupsCollector.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/TopGroupsCollector.java
@@ -70,7 +70,7 @@ public class TopGroupsCollector<T> extends SecondPassGroupingCollector<T> {
this.needsScores = getScores || getMaxScores || withinGroupSort.needsScores();
this.supplier = withinGroupSort == Sort.RELEVANCE ?
() -> TopScoreDocCollector.create(maxDocsPerGroup) :
- () -> TopFieldCollector.create(withinGroupSort, maxDocsPerGroup, fillSortFields, getScores, getMaxScores);
+ () -> TopFieldCollector.create(withinGroupSort, maxDocsPerGroup, fillSortFields, getScores, getMaxScores, true); // TODO: disable exact counts?
}
@Override
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81271141/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java
----------------------------------------------------------------------
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java
index a787e6b..2bc4229 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java
@@ -59,8 +59,6 @@ import org.apache.lucene.index.Term;
import org.apache.lucene.search.BooleanClause.Occur;
import org.apache.lucene.search.BooleanClause;
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.IndexSearcher;
import org.apache.lucene.search.PrefixQuery;
@@ -649,11 +647,7 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
//System.out.println("finalQuery=" + finalQuery);
// Sort by weight, descending:
- 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:
- Collector c2 = new EarlyTerminatingSortingCollector(c, SORT, num);
+ TopFieldCollector c = TopFieldCollector.create(SORT, num, true, false, false, false);
List<LookupResult> results = null;
SearcherManager mgr;
IndexSearcher searcher;
@@ -663,7 +657,7 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
}
try {
//System.out.println("got searcher=" + searcher);
- searcher.search(finalQuery, c2);
+ searcher.search(finalQuery, c);
TopFieldDocs hits = c.topDocs();
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81271141/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java b/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java
index e2bcef8..1c624e1 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java
@@ -537,7 +537,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) : 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);
}
@@ -629,7 +629,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) : 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);
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81271141/solr/core/src/java/org/apache/solr/search/EarlyTerminatingSortingCollector.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/EarlyTerminatingSortingCollector.java b/solr/core/src/java/org/apache/solr/search/EarlyTerminatingSortingCollector.java
new file mode 100644
index 0000000..66f372c
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/search/EarlyTerminatingSortingCollector.java
@@ -0,0 +1,132 @@
+/*
+ * 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.
+ */
+package org.apache.solr.search;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.CollectionTerminatedException;
+import org.apache.lucene.search.Collector;
+import org.apache.lucene.search.FilterCollector;
+import org.apache.lucene.search.FilterLeafCollector;
+import org.apache.lucene.search.LeafCollector;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
+import org.apache.lucene.search.TopDocsCollector;
+import org.apache.lucene.search.TopFieldCollector;
+import org.apache.lucene.search.TotalHitCountCollector;
+
+/**
+ * A {@link Collector} that early terminates collection of documents on a
+ * per-segment basis, if the segment was sorted according to the given
+ * {@link Sort}.
+ *
+ * <p>
+ * <b>NOTE:</b> the {@code Collector} detects segments sorted according to a
+ * an {@link IndexWriterConfig#setIndexSort}. Also, it collects up to a specified
+ * {@code numDocsToCollect} from each segment, and therefore is mostly suitable
+ * for use in conjunction with collectors such as {@link TopDocsCollector}, and
+ * not e.g. {@link TotalHitCountCollector}.
+ * <p>
+ * <b>NOTE</b>: If you wrap a {@code TopDocsCollector} that sorts in the same
+ * order as the index order, the returned {@link TopDocsCollector#topDocs() TopDocs}
+ * will be correct. However the total of {@link TopDocsCollector#getTotalHits()
+ * hit count} will be vastly underestimated since not all matching documents will have
+ * been collected.
+ *
+ * @deprecated Use {@link TopFieldCollector} and set trackTotalHits to false.
+ * @lucene.experimental
+ */
+@Deprecated
+final class EarlyTerminatingSortingCollector extends FilterCollector {
+
+ /** Returns whether collection can be early-terminated if it sorts with the
+ * provided {@link Sort} and if segments are merged with the provided
+ * {@link Sort}. */
+ public static boolean canEarlyTerminate(Sort searchSort, Sort mergePolicySort) {
+ final SortField[] fields1 = searchSort.getSort();
+ final SortField[] fields2 = mergePolicySort.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));
+ }
+
+ /** Sort used to sort the search results */
+ protected final Sort sort;
+ /** Number of documents to collect in each segment */
+ protected final int numDocsToCollect;
+ private final AtomicBoolean terminatedEarly = new AtomicBoolean(false);
+
+ /**
+ * Create a new {@link EarlyTerminatingSortingCollector} instance.
+ *
+ * @param in
+ * the collector to wrap
+ * @param sort
+ * the sort you are sorting the search results on
+ * @param numDocsToCollect
+ * the number of documents to collect on each segment. When wrapping
+ * a {@link TopDocsCollector}, this number should be the number of
+ * hits.
+ * @throws IllegalArgumentException if the sort order doesn't allow for early
+ * termination with the given merge policy.
+ */
+ public EarlyTerminatingSortingCollector(Collector in, Sort sort, int numDocsToCollect) {
+ super(in);
+ if (numDocsToCollect <= 0) {
+ throw new IllegalArgumentException("numDocsToCollect must always be > 0, got " + numDocsToCollect);
+ }
+ this.sort = sort;
+ this.numDocsToCollect = numDocsToCollect;
+ }
+
+ @Override
+ public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
+ Sort segmentSort = context.reader().getMetaData().getSort();
+ if (segmentSort != null && canEarlyTerminate(sort, segmentSort) == false) {
+ throw new IllegalStateException("Cannot early terminate with sort order " + sort + " if segments are sorted with " + segmentSort);
+ }
+
+ if (segmentSort != null) {
+ // segment is sorted, can early-terminate
+ return new FilterLeafCollector(super.getLeafCollector(context)) {
+ private int numCollected;
+
+ @Override
+ public void collect(int doc) throws IOException {
+ super.collect(doc);
+ if (++numCollected >= numDocsToCollect) {
+ terminatedEarly.set(true);
+ throw new CollectionTerminatedException();
+ }
+ }
+
+ };
+ } else {
+ return super.getLeafCollector(context);
+ }
+ }
+
+ public boolean terminatedEarly() {
+ return terminatedEarly.get();
+ }
+}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81271141/solr/core/src/java/org/apache/solr/search/Grouping.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/Grouping.java b/solr/core/src/java/org/apache/solr/search/Grouping.java
index fe781d8..2214a04 100644
--- a/solr/core/src/java/org/apache/solr/search/Grouping.java
+++ b/solr/core/src/java/org/apache/solr/search/Grouping.java
@@ -881,7 +881,7 @@ public class Grouping {
if (sort == null || sort.equals(Sort.RELEVANCE)) {
return TopScoreDocCollector.create(groupDocsToCollect);
} else {
- return TopFieldCollector.create(searcher.weightSort(sort), groupDocsToCollect, false, needScores, needScores);
+ return TopFieldCollector.create(searcher.weightSort(sort), groupDocsToCollect, false, needScores, needScores, true);
}
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81271141/solr/core/src/java/org/apache/solr/search/ReRankCollector.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/ReRankCollector.java b/solr/core/src/java/org/apache/solr/search/ReRankCollector.java
index 8f6bb76..1c0deb1 100644
--- a/solr/core/src/java/org/apache/solr/search/ReRankCollector.java
+++ b/solr/core/src/java/org/apache/solr/search/ReRankCollector.java
@@ -64,7 +64,7 @@ public class ReRankCollector extends TopDocsCollector {
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);
+ this.mainCollector = TopFieldCollector.create(sort, Math.max(this.reRankDocs, length), false, true, true, true);
}
this.searcher = searcher;
this.reRankQueryRescorer = reRankQueryRescorer;
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81271141/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
index 0e8c022..f577579 100644
--- a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
+++ b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
@@ -1518,7 +1518,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
// ... see comments in populateNextCursorMarkFromTopDocs for cache issues (SOLR-5595)
final boolean fillFields = (null != cursor);
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);
}
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81271141/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/QueryCommand.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/QueryCommand.java b/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/QueryCommand.java
index 1615237..e8b61bc 100644
--- a/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/QueryCommand.java
+++ b/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/QueryCommand.java
@@ -127,7 +127,7 @@ public class QueryCommand implements Command<QueryCommandResult> {
if (sort == null || sort.equals(Sort.RELEVANCE)) {
collector = TopScoreDocCollector.create(docsToCollect);
} else {
- collector = TopFieldCollector.create(sort, docsToCollect, true, needScores, needScores);
+ collector = TopFieldCollector.create(sort, docsToCollect, true, needScores, needScores, true);
}
filterCollector = new FilterCollector(docSet, collector);
return Arrays.asList((Collector) filterCollector);
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81271141/solr/core/src/test/org/apache/solr/search/TestSort.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/TestSort.java b/solr/core/src/test/org/apache/solr/search/TestSort.java
index eb3fd00..03365b4 100644
--- a/solr/core/src/test/org/apache/solr/search/TestSort.java
+++ b/solr/core/src/test/org/apache/solr/search/TestSort.java
@@ -285,7 +285,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);
+ final TopFieldCollector topCollector = TopFieldCollector.create(sort, top, true, trackScores, trackMaxScores, true);
final List<MyDoc> collectedDocs = new ArrayList<>();
// delegate and collect docs ourselves