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