You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2016/06/11 15:49:27 UTC

[11/21] lucene-solr:branch_6x: LUCENE-6766: add another random test case; move early terminating collector to core

LUCENE-6766: add another random test case; move early terminating collector to core


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/3010ffac
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/3010ffac
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/3010ffac

Branch: refs/heads/branch_6x
Commit: 3010ffacafd5cc371f4d62413105294d0df37450
Parents: dc5c512
Author: Mike McCandless <mi...@apache.org>
Authored: Sun May 8 07:58:20 2016 -0400
Committer: Mike McCandless <mi...@apache.org>
Committed: Sat Jun 11 11:48:40 2016 -0400

----------------------------------------------------------------------
 .../org/apache/lucene/index/DocIDMerger.java    |   1 +
 .../org/apache/lucene/index/MergeState.java     |   6 +-
 .../org/apache/lucene/index/MultiSorter.java    |   2 +
 .../EarlyTerminatingSortingCollector.java       | 130 +++++++++
 .../apache/lucene/index/TestIndexSorting.java   | 186 +++++++++++++
 .../TestEarlyTerminatingSortingCollector.java   | 262 +++++++++++++++++++
 .../EarlyTerminatingSortingCollector.java       | 132 ----------
 .../TestEarlyTerminatingSortingCollector.java   | 262 -------------------
 8 files changed, 582 insertions(+), 399 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3010ffac/lucene/core/src/java/org/apache/lucene/index/DocIDMerger.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/DocIDMerger.java b/lucene/core/src/java/org/apache/lucene/index/DocIDMerger.java
index 7db7081..7876ebc 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DocIDMerger.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DocIDMerger.java
@@ -58,6 +58,7 @@ public class DocIDMerger<T extends DocIDMerger.Sub> {
     this.subs = subs;
 
     if (indexIsSorted) {
+      // nocommit: we could optimize the (silly) single-sub case and pretend it's a concatenation instead
       queue = new PriorityQueue<T>(maxCount) {
         @Override
         protected boolean lessThan(Sub a, Sub b) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3010ffac/lucene/core/src/java/org/apache/lucene/index/MergeState.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/MergeState.java b/lucene/core/src/java/org/apache/lucene/index/MergeState.java
index 9560983..63eab98 100644
--- a/lucene/core/src/java/org/apache/lucene/index/MergeState.java
+++ b/lucene/core/src/java/org/apache/lucene/index/MergeState.java
@@ -209,20 +209,16 @@ public class MergeState {
     final Sorter sorter = new Sorter(indexSort);
     List<CodecReader> readers = new ArrayList<>(originalReaders.size());
 
-    //System.out.println("MergeState.maybeSortReaders indexSort=" + indexSort);
-
     for (CodecReader leaf : originalReaders) {
       Sort segmentSort = leaf.getIndexSort();
-      //System.out.println("  leaf=" + leaf + " sort=" + segmentSort);
 
       if (segmentSort == null) {
         // TODO: fix IW to also sort when flushing?  It's somewhat tricky because of stored fields and term vectors, which write "live"
-        // to the files on each indexed document:
+        // to their index files on each indexed document:
 
         // This segment was written by flush, so documents are not yet sorted, so we sort them now:
         Sorter.DocMap sortDocMap = sorter.sort(leaf);
         if (sortDocMap != null) {
-          //System.out.println("    sort!");
           leaf = SlowCodecReaderWrapper.wrap(SortingLeafReader.wrap(new MergeReaderWrapper(leaf), sortDocMap));
           leafDocMaps[readers.size()] = new DocMap() {
               @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3010ffac/lucene/core/src/java/org/apache/lucene/index/MultiSorter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/MultiSorter.java b/lucene/core/src/java/org/apache/lucene/index/MultiSorter.java
index 1c67fd5..3448c90 100644
--- a/lucene/core/src/java/org/apache/lucene/index/MultiSorter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/MultiSorter.java
@@ -36,6 +36,8 @@ final class MultiSorter {
    *  documents into the merged segment.  The documents for each incoming leaf reader must already be sorted by the same sort! */
   static MergeState.DocMap[] sort(Sort sort, List<CodecReader> readers) throws IOException {
 
+    // nocommit optimize if only 1 reader is incoming
+
     SortField fields[] = sort.getSort();
     final CrossReaderComparator[] comparators = new CrossReaderComparator[fields.length];
     for(int i=0;i<fields.length;i++) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3010ffac/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
new file mode 100644
index 0000000..1af1b9f
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/EarlyTerminatingSortingCollector.java
@@ -0,0 +1,130 @@
+/*
+ * 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.IndexWriter;
+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().getIndexSort();
+    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/3010ffac/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 3eb30ec..e14606e 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java
@@ -54,13 +54,18 @@ import org.apache.lucene.index.PointValues.Relation;
 import org.apache.lucene.index.TermsEnum.SeekStatus;
 import org.apache.lucene.search.CollectionStatistics;
 import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.EarlyTerminatingSortingCollector;
+import org.apache.lucene.search.FieldDoc;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.Query;
+import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.TermStatistics;
 import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TopFieldCollector;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.Bits;
@@ -1113,4 +1118,185 @@ public class TestIndexSorting extends LuceneTestCase {
     assertReaderEquals("left: sorted by hand; right: sorted by Lucene", r1, r2);
     IOUtils.close(w1, w2, r1, r2, dir1, dir2);
   }
+
+  private static final class RandomDoc {
+    public final int id;
+    public final int intValue;
+    public final long longValue;
+    public final float floatValue;
+    public final double doubleValue;
+    public final byte[] bytesValue;
+    // nocommit postings, points, term vectors
+
+    public RandomDoc(int id) {
+      this.id = id;
+      intValue = random().nextInt();
+      longValue = random().nextLong();
+      floatValue = random().nextFloat();
+      doubleValue = random().nextDouble();
+      bytesValue = new byte[TestUtil.nextInt(random(), 1, 50)];
+      random().nextBytes(bytesValue);
+    }
+  }
+
+  private static Sort randomSort() {
+    int numFields = TestUtil.nextInt(random(), 1, 3);
+    SortField[] sortFields = new SortField[numFields];
+    for(int i=0;i<numFields-1;i++) {
+      boolean reversed = random().nextBoolean();
+      SortField sortField;
+      switch(random().nextInt(5)) {
+      case 0:
+        sortField = new SortField("int", SortField.Type.INT, reversed);
+        if (random().nextBoolean()) {
+          sortField.setMissingValue(random().nextInt());
+        }
+        break;
+      case 1:
+        sortField = new SortField("long", SortField.Type.LONG, reversed);
+        if (random().nextBoolean()) {
+          sortField.setMissingValue(random().nextLong());
+        }
+        break;
+      case 2:
+        sortField = new SortField("float", SortField.Type.FLOAT, reversed);
+        if (random().nextBoolean()) {
+          sortField.setMissingValue(random().nextFloat());
+        }
+        break;
+      case 3:
+        sortField = new SortField("double", SortField.Type.DOUBLE, reversed);
+        if (random().nextBoolean()) {
+          sortField.setMissingValue(random().nextDouble());
+        }
+        break;
+      case 4:
+        sortField = new SortField("bytes", SortField.Type.STRING, reversed);
+        if (random().nextBoolean()) {
+          sortField.setMissingValue(SortField.STRING_LAST);
+        }
+        break;
+      default:
+        throw new AssertionError();
+      }
+      sortFields[i] = sortField;
+    }
+
+    // tie-break by id:
+    sortFields[numFields-1] = new SortField("id", SortField.Type.INT);
+
+    return new Sort(sortFields);
+  }
+
+  // pits index time sorting against query time sorting
+  public void testRandom3() throws Exception {
+    int numDocs;
+    if (TEST_NIGHTLY) {
+      numDocs = atLeast(100000);
+    } else {
+      numDocs = atLeast(1000);
+    }
+    List<RandomDoc> docs = new ArrayList<>();
+
+    Sort sort = randomSort();
+    if (VERBOSE) {
+      System.out.println("TEST: numDocs=" + numDocs + " use sort=" + sort);
+    }
+
+    // no index sorting, all search-time sorting:
+    Directory dir1 = newFSDirectory(createTempDir());
+    IndexWriterConfig iwc1 = newIndexWriterConfig(new MockAnalyzer(random()));
+    IndexWriter w1 = new IndexWriter(dir1, iwc1);
+
+    // use index sorting:
+    Directory dir2 = newFSDirectory(createTempDir());
+    IndexWriterConfig iwc2 = newIndexWriterConfig(new MockAnalyzer(random()));
+    iwc2.setIndexSort(sort);
+    IndexWriter w2 = new IndexWriter(dir2, iwc2);
+
+    for(int id=0;id<numDocs;id++) {
+      RandomDoc docValues = new RandomDoc(id);
+      docs.add(docValues);
+      if (VERBOSE) {
+        System.out.println("TEST: doc id=" + id);
+        System.out.println("  int=" + docValues.intValue);
+        System.out.println("  long=" + docValues.longValue);
+        System.out.println("  float=" + docValues.floatValue);
+        System.out.println("  double=" + docValues.doubleValue);
+        System.out.println("  bytes=" + new BytesRef(docValues.bytesValue));
+      }
+
+      Document doc = new Document();
+      doc.add(new StringField("id", Integer.toString(id), Field.Store.YES));
+      doc.add(new NumericDocValuesField("id", id));
+      doc.add(new NumericDocValuesField("int", docValues.intValue));
+      doc.add(new NumericDocValuesField("long", docValues.longValue));
+      doc.add(new DoubleDocValuesField("double", docValues.doubleValue));
+      doc.add(new FloatDocValuesField("float", docValues.floatValue));
+      doc.add(new SortedDocValuesField("bytes", new BytesRef(docValues.bytesValue)));
+      w1.addDocument(doc);
+      w2.addDocument(doc);
+      // nocommit do some deletions
+    }
+    DirectoryReader r1 = DirectoryReader.open(w1);
+    IndexSearcher s1 = newSearcher(r1);
+
+    if (random().nextBoolean()) {
+      int maxSegmentCount = TestUtil.nextInt(random(), 1, 5);
+      if (VERBOSE) {
+        System.out.println("TEST: now forceMerge(" + maxSegmentCount + ")");
+      }
+      w2.forceMerge(maxSegmentCount);
+    }
+
+    DirectoryReader r2 = DirectoryReader.open(w2);
+    IndexSearcher s2 = newSearcher(r2);
+
+    /*
+    System.out.println("TEST: full index:");
+    SortedDocValues docValues = MultiDocValues.getSortedValues(r2, "bytes");
+    for(int i=0;i<r2.maxDoc();i++) {
+      System.out.println("  doc " + i + " id=" + r2.document(i).get("id") + " bytes=" + docValues.get(i));
+    }
+    */
+
+    for(int iter=0;iter<100;iter++) {
+      int numHits = TestUtil.nextInt(random(), 1, numDocs);
+      if (VERBOSE) {
+        System.out.println("TEST: iter=" + iter + " numHits=" + numHits);
+      }
+
+      TopFieldCollector c1 = TopFieldCollector.create(sort, numHits, 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);
+
+      TopDocs hits2 = c2.topDocs();
+
+      if (VERBOSE) {
+        System.out.println("  topDocs query-time sort: totalHits=" + hits1.totalHits);
+        for(ScoreDoc scoreDoc : hits1.scoreDocs) {
+          System.out.println("    " + scoreDoc.doc);
+        }
+        System.out.println("  topDocs index-time sort: totalHits=" + hits2.totalHits);
+        for(ScoreDoc scoreDoc : hits2.scoreDocs) {
+          System.out.println("    " + scoreDoc.doc);
+        }
+      }
+
+      assertTrue(hits2.totalHits <= hits1.totalHits);
+      assertEquals(hits2.scoreDocs.length, hits1.scoreDocs.length);
+      for(int i=0;i<hits2.scoreDocs.length;i++) {
+        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);
+      }
+    }
+
+    IOUtils.close(r1, r2, w1, w2, dir1, dir2);
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3010ffac/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
new file mode 100644
index 0000000..6108992
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/search/TestEarlyTerminatingSortingCollector.java
@@ -0,0 +1,262 @@
+/*
+ * 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.HashMap;
+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.codecs.Codec;
+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.DirectoryReader;
+import org.apache.lucene.index.ExitableDirectoryReader;
+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.QueryTimeout;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.SerialMergeScheduler;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.LeafCollector;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.search.TopFieldCollector;
+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 final int forceMergeMaxSegmentCount = 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);
+    // nocommit:
+    iwc.setCodec(Codec.forName("SimpleText"));
+    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(forceMergeMaxSegmentCount);
+    }
+    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 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 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/3010ffac/lucene/misc/src/java/org/apache/lucene/search/EarlyTerminatingSortingCollector.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/java/org/apache/lucene/search/EarlyTerminatingSortingCollector.java b/lucene/misc/src/java/org/apache/lucene/search/EarlyTerminatingSortingCollector.java
deleted file mode 100644
index 731c335..0000000
--- a/lucene/misc/src/java/org/apache/lucene/search/EarlyTerminatingSortingCollector.java
+++ /dev/null
@@ -1,132 +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.IndexWriter;
-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
- */
-
-// nocommit move to core too
-
-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().getIndexSort();
-    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/3010ffac/lucene/misc/src/test/org/apache/lucene/search/TestEarlyTerminatingSortingCollector.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/test/org/apache/lucene/search/TestEarlyTerminatingSortingCollector.java b/lucene/misc/src/test/org/apache/lucene/search/TestEarlyTerminatingSortingCollector.java
deleted file mode 100644
index 6108992..0000000
--- a/lucene/misc/src/test/org/apache/lucene/search/TestEarlyTerminatingSortingCollector.java
+++ /dev/null
@@ -1,262 +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.HashMap;
-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.codecs.Codec;
-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.DirectoryReader;
-import org.apache.lucene.index.ExitableDirectoryReader;
-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.QueryTimeout;
-import org.apache.lucene.index.RandomIndexWriter;
-import org.apache.lucene.index.SerialMergeScheduler;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.LeafCollector;
-import org.apache.lucene.search.MatchAllDocsQuery;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.ScoreDoc;
-import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.SortField;
-import org.apache.lucene.search.TermQuery;
-import org.apache.lucene.search.TopFieldCollector;
-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 final int forceMergeMaxSegmentCount = 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);
-    // nocommit:
-    iwc.setCodec(Codec.forName("SimpleText"));
-    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(forceMergeMaxSegmentCount);
-    }
-    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 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 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();
-    }
-  }
-
-}